[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-27 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205747034 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java --- @@ -114,11 +165,77 @@ public void write(StreamSession session,

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-27 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205745293 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java --- @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-26 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205646170 --- Diff: src/java/org/apache/cassandra/db/streaming/ComponentManifest.java --- @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-26 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205639791 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java --- @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-26 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205599465 --- Diff: src/java/org/apache/cassandra/db/streaming/ComponentManifest.java --- @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-26 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205587936 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java --- @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-26 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205586651 --- Diff: src/java/org/apache/cassandra/db/streaming/ComponentManifest.java --- @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-26 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205532095 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java --- @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-26 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205532049 --- Diff: test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java --- @@ -43,8 +51,38 @@ public void serializerTest()

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-26 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205532001 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java --- @@ -183,9 +261,26 @@ public CassandraStreamHeader

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-26 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205504297 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java --- @@ -65,18 +85,43 @@ private CassandraStreamHeader(Version version,

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-26 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205450061 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java --- @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-26 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205445354 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java --- @@ -183,9 +261,26 @@ public CassandraStreamHeader

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-26 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205416649 --- Diff: src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java --- @@ -48,51 +47,61 @@ import

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-26 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205339231 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java --- @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205299016 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java --- @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205298316 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java --- @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205298028 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java --- @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205297956 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java --- @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205297558 --- Diff: src/java/org/apache/cassandra/db/streaming/ComponentManifest.java --- @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205297151 --- Diff: src/java/org/apache/cassandra/db/streaming/ComponentManifest.java --- @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205297184 --- Diff: src/java/org/apache/cassandra/db/streaming/ComponentManifest.java --- @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205283093 --- Diff: src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java --- @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205282807 --- Diff: src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java --- @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205282784 --- Diff: src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java --- @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205277660 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java --- @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205277545 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java --- @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205277492 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java --- @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205276519 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java --- @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205276269 --- Diff: src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java --- @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205276122 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java --- @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205275733 --- Diff: src/java/org/apache/cassandra/db/compaction/Verifier.java --- @@ -361,12 +361,26 @@ public RangeOwnHelper(List> normalizedRanges)

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205275130 --- Diff: src/java/org/apache/cassandra/db/compaction/Verifier.java --- @@ -361,12 +361,26 @@ public RangeOwnHelper(List> normalizedRanges)

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205174576 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java --- @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205175204 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java --- @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205175074 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java --- @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205110785 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java --- @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205109588 --- Diff: src/java/org/apache/cassandra/db/compaction/Verifier.java --- @@ -361,12 +361,26 @@ public RangeOwnHelper(List> normalizedRanges)

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205113409 --- Diff: src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java --- @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205107157 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java --- @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205112542 --- Diff: src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java --- @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205114146 --- Diff: src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java --- @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205113518 --- Diff: src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java --- @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205113992 --- Diff: src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java --- @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205107583 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java --- @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205112000 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java --- @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205113636 --- Diff: src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java --- @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r20523 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java --- @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205108073 --- Diff: src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java --- @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-25 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r204998639 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java --- @@ -62,7 +90,23 @@ public CassandraOutgoingFile(StreamOperation

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-24 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r204955316 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java --- @@ -114,13 +155,51 @@ public void write(StreamSession session,

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-24 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r204955090 --- Diff: src/java/org/apache/cassandra/db/streaming/ComponentInfo.java --- @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-24 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r204955043 --- Diff: src/java/org/apache/cassandra/db/lifecycle/LogFile.java --- @@ -66,7 +66,7 @@ private final LogReplicaSet replicas = new

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-24 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r204954805 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java --- @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-24 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r204954749 --- Diff: src/java/org/apache/cassandra/config/DatabaseDescriptor.java --- @@ -2260,6 +2260,20 @@ public static int getStreamingConnectionsPerHost()

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-24 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r204954729 --- Diff: src/java/org/apache/cassandra/db/streaming/ComponentInfo.java --- @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-24 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r204875677 --- Diff: src/java/org/apache/cassandra/db/streaming/ComponentInfo.java --- @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-24 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r204872490 --- Diff: src/java/org/apache/cassandra/db/lifecycle/LogFile.java --- @@ -66,7 +66,7 @@ private final LogReplicaSet replicas = new

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-24 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r204842484 --- Diff: src/java/org/apache/cassandra/db/lifecycle/LogFile.java --- @@ -66,7 +66,7 @@ private final LogReplicaSet replicas = new

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-24 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r204831709 --- Diff: src/java/org/apache/cassandra/db/lifecycle/LogFile.java --- @@ -66,7 +66,7 @@ private final LogReplicaSet replicas = new

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-23 Thread dineshjoshi
Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r204586299 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java --- @@ -114,13 +155,51 @@ public void write(StreamSession session,

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-23 Thread aweisberg
Github user aweisberg commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r204562307 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java --- @@ -114,13 +155,51 @@ public void write(StreamSession session,

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-23 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r204538045 --- Diff: src/java/org/apache/cassandra/db/streaming/ComponentInfo.java --- @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-23 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r204537184 --- Diff: src/java/org/apache/cassandra/db/streaming/ComponentInfo.java --- @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-23 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r204531887 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java --- @@ -114,13 +155,51 @@ public void write(StreamSession session,

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-23 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r204458523 --- Diff: src/java/org/apache/cassandra/config/DatabaseDescriptor.java --- @@ -2260,6 +2260,20 @@ public static int getStreamingConnectionsPerHost()

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-23 Thread iamaleksey
Github user iamaleksey commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r204437695 --- Diff: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java --- @@ -114,13 +155,51 @@ public void write(StreamSession session,

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-13 Thread dineshjoshi
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,

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-13 Thread dineshjoshi
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

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-13 Thread dineshjoshi
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

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-13 Thread aweisberg
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

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-13 Thread dineshjoshi
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

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-13 Thread dineshjoshi
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

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-13 Thread dineshjoshi
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

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-13 Thread aweisberg
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

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-13 Thread aweisberg
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

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-13 Thread dineshjoshi
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

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-13 Thread aweisberg
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()

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-13 Thread aweisberg
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

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-13 Thread aweisberg
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,

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-13 Thread aweisberg
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()

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-13 Thread aweisberg
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

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-13 Thread aweisberg
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

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-13 Thread dineshjoshi
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

[GitHub] cassandra pull request #239: [CASSANDRA-14556] Optimize Streaming

2018-07-13 Thread aweisberg
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