[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543921#comment-16543921
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user dineshjoshi commented on the issue:

https://github.com/apache/cassandra/pull/239
  
@aweisberg I have updated the dtest to test with different compaction 
strategies and addressed remaining comments.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-14567) CQL query returns different results in 2.2.5 and 3.0.15

2018-07-13 Thread Dikang Gu (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14567?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Dikang Gu updated CASSANDRA-14567:
--
Description: 
During our 2.2.5 to 3.0.15 upgrade, we find the a cql query returns different 
results in 2.2.5 and 3.0.15, here is a unit test to reproduce it, 
[https://gist.github.com/DikangGu/e538ed2de22b74e49b8dd43f7093a996]

 

In C* 2.2.5, it returns all 4 rows: *[Row[1, 10, 0, 1], Row[1, 10, 10, 2], 
Row[1, 1, 0, 3], Row[1, 1, 10, 4]]*

While in C* 3.0.15, it only returns 2 rows to client: *[Row[1, 10, 0, 1], 
Row[1, 1, 10, 4]]*

  

  was:
During our 2.2.5 to 3.0.15 upgrade, we find the a cql query returns different 
results in 2.2 and 3.0, here is a unit test to reproduce it, 
[https://gist.github.com/DikangGu/e538ed2de22b74e49b8dd43f7093a996]

 

In C* 2.2, it returns all 4 rows: *[Row[1, 10, 0, 1], Row[1, 10, 10, 2], Row[1, 
1, 0, 3], Row[1, 1, 10, 4]]*

While in C* 3.0, it only returns 2 rows to client: *[Row[1, 10, 0, 1], Row[1, 
1, 10, 4]]*

  


> CQL query returns different results in 2.2.5 and 3.0.15
> ---
>
> Key: CASSANDRA-14567
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14567
> Project: Cassandra
>  Issue Type: Bug
>  Components: CQL
>Reporter: Dikang Gu
>Priority: Major
>
> During our 2.2.5 to 3.0.15 upgrade, we find the a cql query returns different 
> results in 2.2.5 and 3.0.15, here is a unit test to reproduce it, 
> [https://gist.github.com/DikangGu/e538ed2de22b74e49b8dd43f7093a996]
>  
> In C* 2.2.5, it returns all 4 rows: *[Row[1, 10, 0, 1], Row[1, 10, 10, 2], 
> Row[1, 1, 0, 3], Row[1, 1, 10, 4]]*
> While in C* 3.0.15, it only returns 2 rows to client: *[Row[1, 10, 0, 1], 
> Row[1, 1, 10, 4]]*
>   



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-14567) CQL query returns different results in 2.2.5 and 3.0.15

2018-07-13 Thread Dikang Gu (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14567?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Dikang Gu updated CASSANDRA-14567:
--
Summary: CQL query returns different results in 2.2.5 and 3.0.15  (was: CQL 
query returns different results in 2.2 and 3.0)

> CQL query returns different results in 2.2.5 and 3.0.15
> ---
>
> Key: CASSANDRA-14567
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14567
> Project: Cassandra
>  Issue Type: Bug
>  Components: CQL
>Reporter: Dikang Gu
>Priority: Major
>
> During our 2.2.5 to 3.0.15 upgrade, we find the a cql query returns different 
> results in 2.2 and 3.0, here is a unit test to reproduce it, 
> [https://gist.github.com/DikangGu/e538ed2de22b74e49b8dd43f7093a996]
>  
> In C* 2.2, it returns all 4 rows: *[Row[1, 10, 0, 1], Row[1, 10, 10, 2], 
> Row[1, 1, 0, 3], Row[1, 1, 10, 4]]*
> While in C* 3.0, it only returns 2 rows to client: *[Row[1, 10, 0, 1], Row[1, 
> 1, 10, 4]]*
>   



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14568) Static collection deletions are corrupted in 3.0 -> 2.{1,2} messages

2018-07-13 Thread Benedict (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14568?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543875#comment-16543875
 ] 

Benedict commented on CASSANDRA-14568:
--

Patch available 
[here|https://github.com/belliottsmith/cassandra/tree/CASSANDRA-14568]

> Static collection deletions are corrupted in 3.0 -> 2.{1,2} messages
> 
>
> Key: CASSANDRA-14568
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14568
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Benedict
>Assignee: Benedict
>Priority: Critical
> Fix For: 3.0.18
>
>
> In 2.1 and 2.2, row and complex deletions were represented as range 
> tombstones.  LegacyLayout is our compatibility layer, that translates the 
> relevant RT patterns in 2.1/2.2 to row/complex deletions in 3.0, and vice 
> versa.  Unfortunately, it does not handle the special case of static row 
> deletions, they are treated as regular row deletions. Since static rows are 
> themselves never directly deleted, the only issue is with collection 
> deletions.
> Collection deletions in 2.1/2.2 were encoded as a range tombstone, consisting 
> of a sequence of the clustering keys’ data for the affected row, followed by 
> the bytes representing the name of the collection column.  STATIC_CLUSTERING 
> contains zero clusterings, so by treating the deletion as for a regular row, 
> zero clusterings are written to precede the column name of the erased 
> collection, so the column name is written at position zero.
> This can exhibit itself in at least two ways:
>  # If the type of your first clustering key is a variable width type, new 
> deletes will begin appearing covering the clustering key represented by the 
> column name.
>  ** If you have multiple clustering keys, you will receive a RT covering all 
> those rows with a matching first clustering key.
>  ** This RT will be valid as far as the system is concerned, and go 
> undetected unless there are outside data quality checks in place.
>  # Otherwise, an invalid size of data will be written to the clustering and 
> sent over the network to the 2.1 node.
>  ** The 2.1/2.2 node will handle this just fine, even though the record is 
> junk.  Since it is a deletion covering impossible data, there will be no 
> user-API visible effect.  But if received as a write from a 3.0 node, it will 
> dutifully persist the junk record.
>  ** The 3.0 node that originally sent this junk, may later coordinate a read 
> of the partition, and will notice a digest mismatch, read-repair and 
> serialize the junk to disk
>  ** The sstable containing this record is now corrupt; the deserialization 
> expects fixed-width data, but it encounters too many (or too few) bytes, and 
> is now at an incorrect position to read its structural information
>  ** (Alternatively when the 2.1 node is upgraded this will occur on eventual 
> compaction)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-14568) Static collection deletions are corrupted in 3.0 -> 2.{1,2} messages

2018-07-13 Thread Benedict (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14568?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Benedict updated CASSANDRA-14568:
-
Reviewers: Aleksey Yeschenko

> Static collection deletions are corrupted in 3.0 -> 2.{1,2} messages
> 
>
> Key: CASSANDRA-14568
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14568
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Benedict
>Assignee: Benedict
>Priority: Critical
> Fix For: 3.0.18
>
>
> In 2.1 and 2.2, row and complex deletions were represented as range 
> tombstones.  LegacyLayout is our compatibility layer, that translates the 
> relevant RT patterns in 2.1/2.2 to row/complex deletions in 3.0, and vice 
> versa.  Unfortunately, it does not handle the special case of static row 
> deletions, they are treated as regular row deletions. Since static rows are 
> themselves never directly deleted, the only issue is with collection 
> deletions.
> Collection deletions in 2.1/2.2 were encoded as a range tombstone, consisting 
> of a sequence of the clustering keys’ data for the affected row, followed by 
> the bytes representing the name of the collection column.  STATIC_CLUSTERING 
> contains zero clusterings, so by treating the deletion as for a regular row, 
> zero clusterings are written to precede the column name of the erased 
> collection, so the column name is written at position zero.
> This can exhibit itself in at least two ways:
>  # If the type of your first clustering key is a variable width type, new 
> deletes will begin appearing covering the clustering key represented by the 
> column name.
>  ** If you have multiple clustering keys, you will receive a RT covering all 
> those rows with a matching first clustering key.
>  ** This RT will be valid as far as the system is concerned, and go 
> undetected unless there are outside data quality checks in place.
>  # Otherwise, an invalid size of data will be written to the clustering and 
> sent over the network to the 2.1 node.
>  ** The 2.1/2.2 node will handle this just fine, even though the record is 
> junk.  Since it is a deletion covering impossible data, there will be no 
> user-API visible effect.  But if received as a write from a 3.0 node, it will 
> dutifully persist the junk record.
>  ** The 3.0 node that originally sent this junk, may later coordinate a read 
> of the partition, and will notice a digest mismatch, read-repair and 
> serialize the junk to disk
>  ** The sstable containing this record is now corrupt; the deserialization 
> expects fixed-width data, but it encounters too many (or too few) bytes, and 
> is now at an incorrect position to read its structural information
>  ** (Alternatively when the 2.1 node is upgraded this will occur on eventual 
> compaction)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543873#comment-16543873
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user dineshjoshi commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202493566
  
--- Diff: 
src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java ---
@@ -114,13 +153,54 @@ public void write(StreamSession session, 
DataOutputStreamPlus out, int version)
 CassandraStreamHeader.serializer.serialize(header, out, version);
 out.flush();
 
-CassandraStreamWriter writer = header.compressionInfo == null ?
-   new CassandraStreamWriter(sstable, 
header.sections, session) :
-   new 
CompressedCassandraStreamWriter(sstable, header.sections,
-   
header.compressionInfo, session);
+IStreamWriter writer;
+if (shouldStreamFullSSTable())
+{
+writer = new CassandraBlockStreamWriter(sstable, session, 
components);
+}
+else
+{
+writer = (header.compressionInfo == null) ?
+ new CassandraStreamWriter(sstable, header.sections, 
session) :
+ new CompressedCassandraStreamWriter(sstable, 
header.sections,
+ 
header.compressionInfo, session);
+}
 writer.write(out);
 }
 
+@VisibleForTesting
+public boolean shouldStreamFullSSTable()
+{
+return isFullSSTableTransfersEnabled && isFullyContained;
+}
+
+@VisibleForTesting
+public boolean fullyContainedIn(List> requestedRanges, 
SSTableReader sstable)
+{
+if (requestedRanges == null)
+return false;
+try (KeyIterator iter = new KeyIterator(sstable.descriptor, 
sstable.metadata()))
+{
+while (iter.hasNext())
+{
+DecoratedKey key = iter.next();
+boolean foundFlag = false;
+for (Range r : requestedRanges)
--- End diff --

I refactored it to use `RangeOwnHelper`.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543863#comment-16543863
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user dineshjoshi commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202492216
  
--- Diff: conf/cassandra.yaml ---
@@ -784,6 +784,15 @@ compaction_throughput_mb_per_sec: 16
 # between the sstables, reducing page cache churn and keeping hot rows hot
 sstable_preemptive_open_interval_in_mb: 50
 
+# When enabled, permits Cassandra to zero copy eligible SSTables between
--- End diff --

Updated.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543857#comment-16543857
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user dineshjoshi commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202491699
  
--- Diff: 
src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java ---
@@ -249,4 +250,42 @@ public ByteBufAllocator getAllocator()
 {
 return channelConfig.getAllocator();
 }
+
+/**
+ * Consumes bytes in the stream until the given length
+ *
+ * @param writer
+ * @param len
+ * @return
+ * @throws IOException
+ */
+public long consumeUntil(BufferedDataOutputStreamPlus writer, long 
len) throws IOException
+{
+long copied = 0; // number of bytes copied
+while (copied < len)
+{
+if (buffer.remaining() == 0)
+{
+try
+{
+reBuffer();
+}
+catch (EOFException e)
+{
+throw new EOFException("EOF after " + copied + " bytes 
out of " + len);
+}
+if (buffer.remaining() == 0)
+return copied == 0 ? -1 : copied;
--- End diff --

Ok, makes sense. I have updated it to throw an `AssertionError`.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543852#comment-16543852
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user aweisberg commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202490979
  
--- Diff: 
src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java ---
@@ -249,4 +250,42 @@ public ByteBufAllocator getAllocator()
 {
 return channelConfig.getAllocator();
 }
+
+/**
+ * Consumes bytes in the stream until the given length
+ *
+ * @param writer
+ * @param len
+ * @return
+ * @throws IOException
+ */
+public long consumeUntil(BufferedDataOutputStreamPlus writer, long 
len) throws IOException
+{
+long copied = 0; // number of bytes copied
+while (copied < len)
+{
+if (buffer.remaining() == 0)
+{
+try
+{
+reBuffer();
+}
+catch (EOFException e)
+{
+throw new EOFException("EOF after " + copied + " bytes 
out of " + len);
+}
+if (buffer.remaining() == 0)
+return copied == 0 ? -1 : copied;
--- End diff --

If reBuffer() fails it's going to throw an IOException or RuntimeException 
that won't be caught there so it won't throw AssertionError. It will only throw 
AssertionError when reBuffer() fails to reBuffer() and also doesn't exit via an 
Exception. 

If reBuffer() doesn't reBuffer() and doesn't throw EOF then you don't know 
what's happening. That check doesn't do anything because if it doesn't read the 
require number of bytes it's going to throw EOF (in reBuffer()) and skip it 
anyways.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543835#comment-16543835
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user dineshjoshi commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202489810
  
--- Diff: 
src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java ---
@@ -249,4 +250,42 @@ public ByteBufAllocator getAllocator()
 {
 return channelConfig.getAllocator();
 }
+
+/**
+ * Consumes bytes in the stream until the given length
+ *
+ * @param writer
+ * @param len
+ * @return
+ * @throws IOException
+ */
+public long consumeUntil(BufferedDataOutputStreamPlus writer, long 
len) throws IOException
+{
+long copied = 0; // number of bytes copied
+while (copied < len)
+{
+if (buffer.remaining() == 0)
+{
+try
+{
+reBuffer();
+}
+catch (EOFException e)
+{
+throw new EOFException("EOF after " + copied + " bytes 
out of " + len);
+}
+if (buffer.remaining() == 0)
+return copied == 0 ? -1 : copied;
--- End diff --

Re-reading the code, the check on [line 
227](https://github.com/dineshjoshi/cassandra/blob/f9573ccef63535b787e91e46fa886f3e3ff2eee8/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java#L277)
 is just being over cautious in case the `reBuffer` implementation changes and 
can return a empty buffer.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543834#comment-16543834
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user aweisberg commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202489801
  
--- Diff: conf/cassandra.yaml ---
@@ -784,6 +784,15 @@ compaction_throughput_mb_per_sec: 16
 # between the sstables, reducing page cache churn and keeping hot rows hot
 sstable_preemptive_open_interval_in_mb: 50
 
+# When enabled, permits Cassandra to zero copy eligible SSTables between
--- End diff --

See above.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543829#comment-16543829
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user dineshjoshi commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202488672
  
--- Diff: conf/cassandra.yaml ---
@@ -784,6 +784,15 @@ compaction_throughput_mb_per_sec: 16
 # between the sstables, reducing page cache churn and keeping hot rows hot
 sstable_preemptive_open_interval_in_mb: 50
 
+# When enabled, permits Cassandra to zero copy eligible SSTables between
--- End diff --

3 node cluster with RF=3?


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543827#comment-16543827
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user dineshjoshi commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202488581
  
--- Diff: 
src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java ---
@@ -249,4 +250,42 @@ public ByteBufAllocator getAllocator()
 {
 return channelConfig.getAllocator();
 }
+
+/**
+ * Consumes bytes in the stream until the given length
+ *
+ * @param writer
+ * @param len
+ * @return
+ * @throws IOException
+ */
+public long consumeUntil(BufferedDataOutputStreamPlus writer, long 
len) throws IOException
+{
+long copied = 0; // number of bytes copied
+while (copied < len)
+{
+if (buffer.remaining() == 0)
+{
+try
+{
+reBuffer();
+}
+catch (EOFException e)
+{
+throw new EOFException("EOF after " + copied + " bytes 
out of " + len);
+}
+if (buffer.remaining() == 0)
+return copied == 0 ? -1 : copied;
--- End diff --

Here - 
https://github.com/dineshjoshi/cassandra/blob/f9573ccef63535b787e91e46fa886f3e3ff2eee8/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java#L213

`reBuffer()` could likely fail due to network or channel being closed by 
the other side. It could be treated as a normal `IOException` as we're not 
guaranteed that we would get those many bytes from the other end. It feels a 
bit weird that rest of `RebufferingByteBufDataInputPlus` throws `EOFException` 
and this method would throw an `AssertionError` when there may be a network 
issue.



> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Created] (CASSANDRA-14568) Static collection deletions are corrupted in 3.0 -> 2.{1,2} messages

2018-07-13 Thread Benedict (JIRA)
Benedict created CASSANDRA-14568:


 Summary: Static collection deletions are corrupted in 3.0 -> 
2.{1,2} messages
 Key: CASSANDRA-14568
 URL: https://issues.apache.org/jira/browse/CASSANDRA-14568
 Project: Cassandra
  Issue Type: Bug
Reporter: Benedict
Assignee: Benedict
 Fix For: 3.0.18


In 2.1 and 2.2, row and complex deletions were represented as range tombstones. 
 LegacyLayout is our compatibility layer, that translates the relevant RT 
patterns in 2.1/2.2 to row/complex deletions in 3.0, and vice versa.  
Unfortunately, it does not handle the special case of static row deletions, 
they are treated as regular row deletions. Since static rows are themselves 
never directly deleted, the only issue is with collection deletions.

Collection deletions in 2.1/2.2 were encoded as a range tombstone, consisting 
of a sequence of the clustering keys’ data for the affected row, followed by 
the bytes representing the name of the collection column.  STATIC_CLUSTERING 
contains zero clusterings, so by treating the deletion as for a regular row, 
zero clusterings are written to precede the column name of the erased 
collection, so the column name is written at position zero.

This can exhibit itself in at least two ways:
 # If the type of your first clustering key is a variable width type, new 
deletes will begin appearing covering the clustering key represented by the 
column name.
 ** If you have multiple clustering keys, you will receive a RT covering all 
those rows with a matching first clustering key.
 ** This RT will be valid as far as the system is concerned, and go undetected 
unless there are outside data quality checks in place.
 # Otherwise, an invalid size of data will be written to the clustering and 
sent over the network to the 2.1 node.
 ** The 2.1/2.2 node will handle this just fine, even though the record is 
junk.  Since it is a deletion covering impossible data, there will be no 
user-API visible effect.  But if received as a write from a 3.0 node, it will 
dutifully persist the junk record.
 ** The 3.0 node that originally sent this junk, may later coordinate a read of 
the partition, and will notice a digest mismatch, read-repair and serialize the 
junk to disk
 ** The sstable containing this record is now corrupt; the deserialization 
expects fixed-width data, but it encounters too many (or too few) bytes, and is 
now at an incorrect position to read its structural information
 ** (Alternatively when the 2.1 node is upgraded this will occur on eventual 
compaction)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread Ariel Weisberg (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543819#comment-16543819
 ] 

Ariel Weisberg commented on CASSANDRA-14556:


This is really close. The main thing is just that the dtest doesn't highlight 
the scenarios where we want this optimization to work and then check it is 
effective.

> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543818#comment-16543818
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user aweisberg commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202487364
  
--- Diff: conf/cassandra.yaml ---
@@ -784,6 +784,15 @@ compaction_throughput_mb_per_sec: 16
 # between the sstables, reducing page cache churn and keeping hot rows hot
 sstable_preemptive_open_interval_in_mb: 50
 
+# When enabled, permits Cassandra to zero copy eligible SSTables between
--- End diff --

Except for the use cases where you have a small number of nodes, but that 
is a pretty big caveat.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543817#comment-16543817
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user aweisberg commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202487302
  
--- Diff: conf/cassandra.yaml ---
@@ -784,6 +784,15 @@ compaction_throughput_mb_per_sec: 16
 # between the sstables, reducing page cache churn and keeping hot rows hot
 sstable_preemptive_open_interval_in_mb: 50
 
+# When enabled, permits Cassandra to zero copy eligible SSTables between
--- End diff --

I don't see how STCS can possibly benefit without 6696. It will always 
always have keys spanning all the ranges owned by that node in it.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543812#comment-16543812
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user dineshjoshi commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202487016
  
--- Diff: conf/cassandra.yaml ---
@@ -784,6 +784,15 @@ compaction_throughput_mb_per_sec: 16
 # between the sstables, reducing page cache churn and keeping hot rows hot
 sstable_preemptive_open_interval_in_mb: 50
 
+# When enabled, permits Cassandra to zero copy eligible SSTables between
--- End diff --

Leveled - yes, STCS - partly, 6696 should have a positive impact regardless 
of the strategy.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543810#comment-16543810
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user aweisberg commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202486943
  
--- Diff: 
src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java ---
@@ -249,4 +250,42 @@ public ByteBufAllocator getAllocator()
 {
 return channelConfig.getAllocator();
 }
+
+/**
+ * Consumes bytes in the stream until the given length
+ *
+ * @param writer
+ * @param len
+ * @return
+ * @throws IOException
+ */
+public long consumeUntil(BufferedDataOutputStreamPlus writer, long 
len) throws IOException
+{
+long copied = 0; // number of bytes copied
+while (copied < len)
+{
+if (buffer.remaining() == 0)
+{
+try
+{
+reBuffer();
+}
+catch (EOFException e)
+{
+throw new EOFException("EOF after " + copied + " bytes 
out of " + len);
+}
+if (buffer.remaining() == 0)
+return copied == 0 ? -1 : copied;
--- End diff --

So reBuffer seems to have as its contract that it will throw EOF? That's 
why if you reach this point it is AssertionError since you don't know why 
reBuffer messed up.

Which check in BigTableBlockWriter are you talking about?


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543805#comment-16543805
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user aweisberg commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202486275
  
--- Diff: conf/cassandra.yaml ---
@@ -784,6 +784,15 @@ compaction_throughput_mb_per_sec: 16
 # between the sstables, reducing page cache churn and keeping hot rows hot
 sstable_preemptive_open_interval_in_mb: 50
 
+# When enabled, permits Cassandra to zero copy eligible SSTables between
--- End diff --

Worth documenting this optimization relies on having sstables that are 
wholly contained in the range being streamed and that configurations like size 
tiered without 6696 won't benefit, and would in fact spend time reading indexes 
checking to see if the optimization applies? Leveled should always benefit from 
this optimization.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543802#comment-16543802
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user aweisberg commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202486107
  
--- Diff: 
src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java ---
@@ -114,13 +153,54 @@ public void write(StreamSession session, 
DataOutputStreamPlus out, int version)
 CassandraStreamHeader.serializer.serialize(header, out, version);
 out.flush();
 
-CassandraStreamWriter writer = header.compressionInfo == null ?
-   new CassandraStreamWriter(sstable, 
header.sections, session) :
-   new 
CompressedCassandraStreamWriter(sstable, header.sections,
-   
header.compressionInfo, session);
+IStreamWriter writer;
+if (shouldStreamFullSSTable())
+{
+writer = new CassandraBlockStreamWriter(sstable, session, 
components);
+}
+else
+{
+writer = (header.compressionInfo == null) ?
+ new CassandraStreamWriter(sstable, header.sections, 
session) :
+ new CompressedCassandraStreamWriter(sstable, 
header.sections,
+ 
header.compressionInfo, session);
+}
 writer.write(out);
 }
 
+@VisibleForTesting
+public boolean shouldStreamFullSSTable()
+{
+return isFullSSTableTransfersEnabled && isFullyContained;
+}
+
+@VisibleForTesting
+public boolean fullyContainedIn(List> requestedRanges, 
SSTableReader sstable)
+{
+if (requestedRanges == null)
+return false;
+try (KeyIterator iter = new KeyIterator(sstable.descriptor, 
sstable.metadata()))
+{
+while (iter.hasNext())
+{
+DecoratedKey key = iter.next();
+boolean foundFlag = false;
+for (Range r : requestedRanges)
--- End diff --

So in the vnodes case is this going to be 256 ranges by default? Should 
this be a binary search instead of a scan?


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543798#comment-16543798
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user dineshjoshi commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202485483
  
--- Diff: 
src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java ---
@@ -249,4 +250,42 @@ public ByteBufAllocator getAllocator()
 {
 return channelConfig.getAllocator();
 }
+
+/**
+ * Consumes bytes in the stream until the given length
+ *
+ * @param writer
+ * @param len
+ * @return
+ * @throws IOException
+ */
+public long consumeUntil(BufferedDataOutputStreamPlus writer, long 
len) throws IOException
+{
+long copied = 0; // number of bytes copied
+while (copied < len)
+{
+if (buffer.remaining() == 0)
+{
+try
+{
+reBuffer();
+}
+catch (EOFException e)
+{
+throw new EOFException("EOF after " + copied + " bytes 
out of " + len);
+}
+if (buffer.remaining() == 0)
+return copied == 0 ? -1 : copied;
--- End diff --

Right now the check is being made in `BigTableBlockWriter` and it throws an 
`IOException`, do we need to convert it into an `AssertionError`? We could 
convert this into an `EOFException` which is more uniform with the way the rest 
of the class handles things. WDYT?


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-14567) CQL query returns different results in 2.2 and 3.0

2018-07-13 Thread Dikang Gu (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14567?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Dikang Gu updated CASSANDRA-14567:
--
Description: 
During our 2.2.5 to 3.0.15 upgrade, we find the a cql query returns different 
results in 2.2 and 3.0, here is a unit test to reproduce it, 
[https://gist.github.com/DikangGu/e538ed2de22b74e49b8dd43f7093a996]

 

In C* 2.2, it returns all 4 rows: *[Row[1, 10, 0, 1], Row[1, 10, 10, 2], Row[1, 
1, 0, 3], Row[1, 1, 10, 4]]*

While in C* 3.0, it only returns 2 rows to client: *[Row[1, 10, 0, 1], Row[1, 
1, 10, 4]]*

  

  was:
During our 2.2.5 to 3.0.15 upgrade, we find the a cql query returns different 
results in 2.2 and 3.0, here is a unit test to reproduce it, 
[https://gist.github.com/DikangGu/e538ed2de22b74e49b8dd43f7093a996]

 

In C* 2.2, it returns all 4 rows: *[Row[1, 10, 0, 1], Row[1, 10, 10, 2], Row[1, 
1, 0, 3], Row[1, 1, 10, 4]]*

While in C* 3.0, it only returns 2 rows client: *[Row[1, 10, 0, 1], Row[1, 1, 
10, 4]]*

  


> CQL query returns different results in 2.2 and 3.0
> --
>
> Key: CASSANDRA-14567
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14567
> Project: Cassandra
>  Issue Type: Bug
>  Components: CQL
>Reporter: Dikang Gu
>Priority: Major
>
> During our 2.2.5 to 3.0.15 upgrade, we find the a cql query returns different 
> results in 2.2 and 3.0, here is a unit test to reproduce it, 
> [https://gist.github.com/DikangGu/e538ed2de22b74e49b8dd43f7093a996]
>  
> In C* 2.2, it returns all 4 rows: *[Row[1, 10, 0, 1], Row[1, 10, 10, 2], 
> Row[1, 1, 0, 3], Row[1, 1, 10, 4]]*
> While in C* 3.0, it only returns 2 rows to client: *[Row[1, 10, 0, 1], Row[1, 
> 1, 10, 4]]*
>   



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543778#comment-16543778
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user aweisberg commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202481801
  
--- Diff: 
src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java ---
@@ -249,4 +250,42 @@ public ByteBufAllocator getAllocator()
 {
 return channelConfig.getAllocator();
 }
+
+/**
+ * Consumes bytes in the stream until the given length
+ *
+ * @param writer
+ * @param len
+ * @return
+ * @throws IOException
+ */
+public long consumeUntil(BufferedDataOutputStreamPlus writer, long 
len) throws IOException
+{
+long copied = 0; // number of bytes copied
+while (copied < len)
+{
+if (buffer.remaining() == 0)
+{
+try
+{
+reBuffer();
+}
+catch (EOFException e)
+{
+throw new EOFException("EOF after " + copied + " bytes 
out of " + len);
+}
+if (buffer.remaining() == 0)
+return copied == 0 ? -1 : copied;
--- End diff --

Heh, sorry I'm back for more. So reBuffering always does this check. I 
would change this to throw AssertionError because if it happens we don't 
actually know if it's end of stream or not because reBuffer didn't do what 
reBuffer is supposed to do.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543768#comment-16543768
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user aweisberg commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202480493
  
--- Diff: 
src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java ---
@@ -168,10 +232,35 @@ public void serialize(CassandraStreamHeader header, 
DataOutputPlus out, int vers
 header.calculateCompressionInfo();
 CompressionInfo.serializer.serialize(header.compressionInfo, 
out, version);
 out.writeInt(header.sstableLevel);
+
 SerializationHeader.serializer.serialize(header.version, 
header.header, out);
+
+header.tableId.serialize(out);
+out.writeBoolean(header.fullStream);
+
+if (header.fullStream)
+{
+out.writeInt(header.components.size());
+for (ComponentInfo info : header.components)
+ComponentInfo.serializer.serialize(info, out, version);
+
+
ByteBufferUtil.writeWithShortLength(header.firstKey.getKey(), out);
--- End diff --

OK, actually Jeff pointed out that we do have partitioners where decorated 
keys are not hashes. So either we find out what the max length is or just use 
an int/varint and call it good. We don't send this often so it's fine.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-14567) CQL query returns different results in 2.2 and 3.0

2018-07-13 Thread Dikang Gu (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14567?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Dikang Gu updated CASSANDRA-14567:
--
Description: 
During our 2.2.5 to 3.0.15 upgrade, we find the a cql query returns different 
results in 2.2 and 3.0, here is a unit test to reproduce it, 
[https://gist.github.com/DikangGu/e538ed2de22b74e49b8dd43f7093a996]

 

In C* 2.2, it returns all 4 rows: *[Row[1, 10, 0, 1], Row[1, 10, 10, 2], Row[1, 
1, 0, 3], Row[1, 1, 10, 4]]*

While in C* 3.0, it only returns 2 rows client: *[Row[1, 10, 0, 1], Row[1, 1, 
10, 4]]*

  

  was:
During our 2.2.5 to 3.0.15 upgrade, we find the a cql query returns different 
results in 2.2 and 3.0, here is a unit test to reproduce it, 
[https://gist.github.com/DikangGu/e538ed2de22b74e49b8dd43f7093a996]

 

In C* 2.2, it returns all 4 rows: *[Row[1, 10, 0, 1], Row[1, 10, 10, 2], Row[1, 
1, 0, 3], Row[1, 1, 10, 4]]*

While in C* 3.0, it only returns 2 row client: *[Row[1, 10, 0, 1], Row[1, 1, 
10, 4]]*

  


> CQL query returns different results in 2.2 and 3.0
> --
>
> Key: CASSANDRA-14567
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14567
> Project: Cassandra
>  Issue Type: Bug
>  Components: CQL
>Reporter: Dikang Gu
>Priority: Major
>
> During our 2.2.5 to 3.0.15 upgrade, we find the a cql query returns different 
> results in 2.2 and 3.0, here is a unit test to reproduce it, 
> [https://gist.github.com/DikangGu/e538ed2de22b74e49b8dd43f7093a996]
>  
> In C* 2.2, it returns all 4 rows: *[Row[1, 10, 0, 1], Row[1, 10, 10, 2], 
> Row[1, 1, 0, 3], Row[1, 1, 10, 4]]*
> While in C* 3.0, it only returns 2 rows client: *[Row[1, 10, 0, 1], Row[1, 1, 
> 10, 4]]*
>   



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Created] (CASSANDRA-14567) CQL query returns different results in 2.2 and 3.0

2018-07-13 Thread Dikang Gu (JIRA)
Dikang Gu created CASSANDRA-14567:
-

 Summary: CQL query returns different results in 2.2 and 3.0
 Key: CASSANDRA-14567
 URL: https://issues.apache.org/jira/browse/CASSANDRA-14567
 Project: Cassandra
  Issue Type: Bug
  Components: CQL
Reporter: Dikang Gu


During our 2.2.5 to 3.0.15 upgrade, we find the a cql query returns different 
results in 2.2 and 3.0, here is a unit test to reproduce it, 
[https://gist.github.com/DikangGu/e538ed2de22b74e49b8dd43f7093a996]

 

In C* 2.2, it returns all 4 rows: *[Row[1, 10, 0, 1], Row[1, 10, 10, 2], Row[1, 
1, 0, 3], Row[1, 1, 10, 4]]*

While in C* 3.0, it only returns 2 row client: *[Row[1, 10, 0, 1], Row[1, 1, 
10, 4]]*

  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543754#comment-16543754
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user aweisberg commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202478981
  
--- Diff: 
src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java ---
@@ -168,10 +232,35 @@ public void serialize(CassandraStreamHeader header, 
DataOutputPlus out, int vers
 header.calculateCompressionInfo();
 CompressionInfo.serializer.serialize(header.compressionInfo, 
out, version);
 out.writeInt(header.sstableLevel);
+
 SerializationHeader.serializer.serialize(header.version, 
header.header, out);
+
+header.tableId.serialize(out);
+out.writeBoolean(header.fullStream);
+
+if (header.fullStream)
+{
+out.writeInt(header.components.size());
+for (ComponentInfo info : header.components)
+ComponentInfo.serializer.serialize(info, out, version);
+
+
ByteBufferUtil.writeWithShortLength(header.firstKey.getKey(), out);
--- End diff --

It's fine I forgot that the keys are hashes so they will always be fixed 
size and small.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543742#comment-16543742
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user dineshjoshi commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202477596
  
--- Diff: 
src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java ---
@@ -211,12 +228,16 @@ public void serialize(CassandraStreamHeader header, 
DataOutputPlus out, int vers
 
 SerializationHeader.serializer.serialize(header.version, 
header.header, out);
 
+header.tableId.serialize(out);
 out.writeBoolean(header.fullStream);
+
 if (header.fullStream)
 {
 out.writeInt(header.components.size());
 for (ComponentInfo info : header.components)
 ComponentInfo.serializer.serialize(info, out, version);
+
+
ByteBufferUtil.writeWithShortLength(header.firstKey.getKey(), out);
--- End diff --

Yes, this is the first situation where we're actually serializing 
`DecoratedKeys`. I am not sure if it'll overflow.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543739#comment-16543739
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user dineshjoshi commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202477311
  
--- Diff: src/java/org/apache/cassandra/db/DiskBoundaries.java ---
@@ -129,4 +129,19 @@ public int 
getBoundariesFromSSTableDirectory(Descriptor descriptor)
 {
 return directories.get(getDiskIndex(sstable));
 }
+
+public Directories.DataDirectory getCorrectDiskForKey(DecoratedKey key)
+{
+if (positions == null)
+return null;
+
+return directories.get(getDiskIndex(key));
+}
+
+private int getDiskIndex(DecoratedKey key)
+{
+int pos = Collections.binarySearch(positions, key);
--- End diff --

This method is pretty much a replica of `getDiskIndex(SSTableReader 
sstable)` (See: 
https://github.com/dineshjoshi/cassandra/blob/657ce3670a5291decf2939d72d51abe19333ef15/src/java/org/apache/cassandra/db/DiskBoundaries.java#L102)
 Instead of accepting a sstable, I pass in a `DecoratedKey`.

We ensure that the range is owned at the upper levels of the stack.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543738#comment-16543738
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user aweisberg commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202477083
  
--- Diff: 
src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java ---
@@ -211,12 +228,16 @@ public void serialize(CassandraStreamHeader header, 
DataOutputPlus out, int vers
 
 SerializationHeader.serializer.serialize(header.version, 
header.header, out);
 
+header.tableId.serialize(out);
 out.writeBoolean(header.fullStream);
+
 if (header.fullStream)
 {
 out.writeInt(header.components.size());
 for (ComponentInfo info : header.components)
 ComponentInfo.serializer.serialize(info, out, version);
+
+
ByteBufferUtil.writeWithShortLength(header.firstKey.getKey(), out);
--- End diff --

I guess decorated keys don't get bigger than that? I mean... I couldn't 
find us directly serializing decorated keys anywhere else. I'm not sure.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543733#comment-16543733
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user aweisberg commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/239#discussion_r202476142
  
--- Diff: src/java/org/apache/cassandra/db/DiskBoundaries.java ---
@@ -129,4 +129,19 @@ public int 
getBoundariesFromSSTableDirectory(Descriptor descriptor)
 {
 return directories.get(getDiskIndex(sstable));
 }
+
+public Directories.DataDirectory getCorrectDiskForKey(DecoratedKey key)
+{
+if (positions == null)
+return null;
+
+return directories.get(getDiskIndex(key));
+}
+
+private int getDiskIndex(DecoratedKey key)
+{
+int pos = Collections.binarySearch(positions, key);
--- End diff --

OK so this is insertion point, and then -1 because the of 0 based indexing? 
Can you unit test this just to make sure that it find the correct spot?

I'm just not following the -1 I guess because Collections.binarySearch says 
insertion point -1 is what is returned. What if the key happens to be out of 
range of the ranges this node owns? This would go out of bounds?


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543646#comment-16543646
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user aweisberg commented on the issue:

https://github.com/apache/cassandra/pull/239
  
Ah it's the log statement block stream? Great. I think the thing I would 
look for then is just some log details about what percentage was block streamed 
vs regular streamed and assert that the expected amount of block streaming 
occurred.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543606#comment-16543606
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user dineshjoshi commented on the issue:

https://github.com/apache/cassandra/pull/239
  
The dtest does validate the kind of streaming occurred see - 
https://github.com/apache/cassandra-dtest/compare/master...dineshjoshi:faster-streaming-rev2#diff-556858bf3074903213f73e382cf70297R58


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543602#comment-16543602
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user aweisberg commented on the issue:

https://github.com/apache/cassandra/pull/239
  
The dtest doesn't validate that the kind of streaming you wanted to occur 
occured just that streaming succeeded.

I would also like to see it validate that if you run leveled compaction it 
will tend stream whole tables from the deeper levels which is one of the 
success criteria.

It would also be good to see that it works with CASSANDRA-6696 and size 
tiered and ends up streaming whole sstables.


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14556) Optimize streaming path in Cassandra

2018-07-13 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16543505#comment-16543505
 ] 

ASF GitHub Bot commented on CASSANDRA-14556:


Github user dineshjoshi commented on the issue:

https://github.com/apache/cassandra/pull/239
  
1. Fixed the data directory selection
2. Added & updated unit tests
3. Added documentation in cassandra.yaml
4. Updated the microbenchmark to benchmark the read and write path for 
faster streaming as well as existing streaming


> Optimize streaming path in Cassandra
> 
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Streaming and Messaging
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-14566) Cache CSM.onlyPurgeRepairedTombstones()

2018-07-13 Thread Stefan Podkowinski (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14566?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Stefan Podkowinski updated CASSANDRA-14566:
---
Labels: lhf performance  (was: performance)

> Cache CSM.onlyPurgeRepairedTombstones()
> ---
>
> Key: CASSANDRA-14566
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14566
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Stefan Podkowinski
>Priority: Minor
>  Labels: lhf, performance
>
> We currently call {{CompactionStrategyManager.onlyPurgeRepairedTombstones()}} 
> *a lot* during compaction, I think at least for every key. So we should 
> probably cache the value, instead of constantly reading from a volatile and 
> calling parseBoolean.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13938) Default repair is broken, crashes other nodes participating in repair (in trunk)

2018-07-13 Thread Dimitar Dimitrov (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-13938?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16542961#comment-16542961
 ] 

Dimitar Dimitrov commented on CASSANDRA-13938:
--

{quote}The problem is that when {{CompressedInputStream#position()}} is called, 
the new position might be in the middle of a buffer. We need to remember that 
offset, and subtract that value when updating {{current}} in 
{{#reBuffer(boolean)}}. The resaon why is that those offset bytes get double 
counted on the first call to {{#reBuffer()}} after {{#position()}} as we add 
the {{buffer.position()}} to {{current}}. {{current}} already accounts for 
those offset bytes when {{#position()}} was called.
{quote}
[~jasobrown], isn't that equivalent (although a bit more complex) to just 
setting {{current}} to the last reached/read position in the stream when 
rebuffering? (i.e. {{current = streamOffset + buffer.position()}}).

I might be missing something, but the role of {{currentBufferOffset}} seems to 
be solely to "align" {{current}} and {{streamOffset}} the first time after a 
new section is started. Then {{current += buffer.position() - 
currentBufferOffse expands to }}{{current = -current- + buffer.position() + 
streamOffset - -current- }}which is the same as {{current = streamOffset + 
buffer.position()}}. After that first time, {{current}} naturally follows 
{{streamOffset}} without the need of any adjustment, but it seems more natural 
to express this as {{streamOffset + buffer.position()}} instead of the new 
expression or the old {{current + buffer.position()}}. To me, it's also a bit 
more intuitive and easier to understand (hopefully it's also right in addition 
to intuitive :)).

The equivalence above would hold true if {{current}} and {{streamOffset}} don't 
change their value in the meantime, but I think this is ensured by the 
well-ordered sequential fashion in which the decompressing and the offset 
bookkeeping functionality of {{CompressedInputStream}} happen in the thread 
running the corresponding {{StreamDeserializingTask}}.
 * The aforementioned well-ordered sequential fashion seems to be POSITION 
followed by 0-N times REBUFFER + DECOMPRESS, where the first REBUFFER might not 
update {{current}} with the above calculation in case {{current}} is already 
too far ahead (i.e. the new section is not starting within the current buffer).

> Default repair is broken, crashes other nodes participating in repair (in 
> trunk)
> 
>
> Key: CASSANDRA-13938
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13938
> Project: Cassandra
>  Issue Type: Bug
>  Components: Repair
>Reporter: Nate McCall
>Assignee: Jason Brown
>Priority: Critical
> Fix For: 4.x
>
> Attachments: 13938.yaml, test.sh
>
>
> Running through a simple scenario to test some of the new repair features, I 
> was not able to make a repair command work. Further, the exception seemed to 
> trigger a nasty failure state that basically shuts down the netty connections 
> for messaging *and* CQL on the nodes transferring back data to the node being 
> repaired. The following steps reproduce this issue consistently.
> Cassandra stress profile (probably not necessary, but this one provides a 
> really simple schema and consistent data shape):
> {noformat}
> keyspace: standard_long
> keyspace_definition: |
>   CREATE KEYSPACE standard_long WITH replication = {'class':'SimpleStrategy', 
> 'replication_factor':3};
> table: test_data
> table_definition: |
>   CREATE TABLE test_data (
>   key text,
>   ts bigint,
>   val text,
>   PRIMARY KEY (key, ts)
>   ) WITH COMPACT STORAGE AND
>   CLUSTERING ORDER BY (ts DESC) AND
>   bloom_filter_fp_chance=0.01 AND
>   caching={'keys':'ALL', 'rows_per_partition':'NONE'} AND
>   comment='' AND
>   dclocal_read_repair_chance=0.00 AND
>   gc_grace_seconds=864000 AND
>   read_repair_chance=0.00 AND
>   compaction={'class': 'SizeTieredCompactionStrategy'} AND
>   compression={'sstable_compression': 'LZ4Compressor'};
> columnspec:
>   - name: key
> population: uniform(1..5000) # 50 million records available
>   - name: ts
> cluster: gaussian(1..50) # Up to 50 inserts per record
>   - name: val
> population: gaussian(128..1024) # varrying size of value data
> insert:
>   partitions: fixed(1) # only one insert per batch for individual partitions
>   select: fixed(1)/1 # each insert comes in one at a time
>   batchtype: UNLOGGED
> queries:
>   single:
> cql: select * from test_data where key = ? and ts = ? limit 1;
>   series:
> cql: select key,ts,val from test_data where key = ? limit 10;
> {noformat}
> The commands to build and run:
> {noformat}
> ccm create 4_0_test -v git:trunk -n 3 -s
> ccm stress user profi

[jira] [Created] (CASSANDRA-14566) Cache CSM.onlyPurgeRepairedTombstones()

2018-07-13 Thread Stefan Podkowinski (JIRA)
Stefan Podkowinski created CASSANDRA-14566:
--

 Summary: Cache CSM.onlyPurgeRepairedTombstones()
 Key: CASSANDRA-14566
 URL: https://issues.apache.org/jira/browse/CASSANDRA-14566
 Project: Cassandra
  Issue Type: Improvement
  Components: Compaction
Reporter: Stefan Podkowinski


We currently call {{CompactionStrategyManager.onlyPurgeRepairedTombstones()}} 
*a lot* during compaction, I think at least for every key. So we should 
probably cache the value, instead of constantly reading from a volatile and 
calling parseBoolean.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14204) Nodetool garbagecollect AssertionError

2018-07-13 Thread Stefan Podkowinski (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14204?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16542907#comment-16542907
 ] 

Stefan Podkowinski commented on CASSANDRA-14204:


So the patched implementation pretty much follows {{performSSTableRewrite()}}, 
which looks like the correct way to handle this to me. We could modify 
{{GcCompactionTest}} a bit to make some of the effects more testable, see 
[ebd7de7|https://github.com/spodkowinski/cassandra/commit/ebd7de758b48a6f924d60eeecbc615c355c87257].

> Nodetool garbagecollect AssertionError
> --
>
> Key: CASSANDRA-14204
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14204
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
>Reporter: Vincent White
>Assignee: Vincent White
>Priority: Minor
> Fix For: 3.11.x, 4.x
>
>
> When manually running a garbage collection compaction across a table with 
> unrepaired sstables and only_purge_repaired_tombstones set to true an 
> assertion error is thrown. This is because the unrepaired sstables aren't 
> being removed from the transaction as they are filtered out in 
> filterSSTables().
> ||3.11||trunk||
> |[branch|https://github.com/vincewhite/cassandra/commit/e13c822736edd3df3403c02e8ef90816f158cde2]|[branch|https://github.com/vincewhite/cassandra/commit/cc8828576404e72504d9b334be85f84c90e77aa7]|
> The stacktrace:
> {noformat}
> -- StackTrace --
> java.lang.AssertionError
>   at 
> org.apache.cassandra.db.compaction.CompactionManager.parallelAllSSTableOperation(CompactionManager.java:339)
>   at 
> org.apache.cassandra.db.compaction.CompactionManager.performGarbageCollection(CompactionManager.java:476)
>   at 
> org.apache.cassandra.db.ColumnFamilyStore.garbageCollect(ColumnFamilyStore.java:1579)
>   at 
> org.apache.cassandra.service.StorageService.garbageCollect(StorageService.java:3069)
>   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>   at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>   at java.lang.reflect.Method.invoke(Method.java:498)
>   at sun.reflect.misc.Trampoline.invoke(MethodUtil.java:71)
>   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>   at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>   at java.lang.reflect.Method.invoke(Method.java:498)
>   at sun.reflect.misc.MethodUtil.invoke(MethodUtil.java:275)
>   at 
> com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:112)
>   at 
> com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:46)
>   at 
> com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:237)
>   at com.sun.jmx.mbeanserver.PerInterface.invoke(PerInterface.java:138)
>   at com.sun.jmx.mbeanserver.MBeanSupport.invoke(MBeanSupport.java:252)
>   at 
> com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.invoke(DefaultMBeanServerInterceptor.java:819)
>   at 
> com.sun.jmx.mbeanserver.JmxMBeanServer.invoke(JmxMBeanServer.java:801)
>   at 
> javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1468)
>   at 
> javax.management.remote.rmi.RMIConnectionImpl.access$300(RMIConnectionImpl.java:76)
>   at 
> javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1309)
>   at 
> javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1401)
>   at 
> javax.management.remote.rmi.RMIConnectionImpl.invoke(RMIConnectionImpl.java:829)
>   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>   at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>   at java.lang.reflect.Method.invoke(Method.java:498)
>   at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:357)
>   at sun.rmi.transport.Transport$1.run(Transport.java:200)
>   at sun.rmi.transport.Transport$1.run(Transport.java:197)
>   at java.security.AccessController.doPrivileged(Native Method)
>   at sun.rmi.transport.Transport.serviceCall(Transport.java:196)
>   at 
> sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:568)
>   at 
> sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:826)
>   at 
> sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.lambda$run$0(TCPTransp

[jira] [Commented] (CASSANDRA-14289) Document sstable tools

2018-07-13 Thread Valerie Parham-Thompson (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14289?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16542869#comment-16542869
 ] 

Valerie Parham-Thompson commented on CASSANDRA-14289:
-

Yes, I will commit to this if you would like to assign it, [~jjirsa]. Thank you!

> Document sstable tools
> --
>
> Key: CASSANDRA-14289
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14289
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Documentation and Website
>Reporter: Hannu Kröger
>Priority: Major
>
> Following tools are missing in the documentation of cassandra tools on the 
> documentation site (http://cassandra.apache.org/doc/latest/tools/index.html):
>  * sstabledump
>  * sstableexpiredblockers
>  * sstablelevelreset
>  * sstableloader
>  * sstablemetadata
>  * sstableofflinerelevel
>  * sstablerepairedset
>  * sstablescrub
>  * sstablesplit
>  * sstableupgrade
>  * sstableutil
>  * sstableverify



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org