[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726901#comment-14726901 ] David Kua commented on CASSANDRA-9304: -- [~Stefania] Updates to my 9304 branch now include a parameter for the COPY command that allows for number of jobs to be configured. RateMeter was also changed and fixed up as an issue was found during testing. Testing also found issues with ByteOrderedPartitioner and OrderPreservingPartitioner. Mainly that BOP's tokens don't work with the SELECT statements I'm using and OPP has no token ring so can't be parallelized. So changes were made to cause COPY TO to run as if it were single process when it encounters those two partitioners. Tests were updated and can be found here: https://github.com/dkua/cassandra-dtest/tree/bulk_export The cqlsh COPY tests now run with a cluster of 3 nodes and the tests have increased from testing 1k rows to 10k rows. One of the read/write tests now tests different partitioners also and should cover that case perfectly fine. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: David Kua >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14728422#comment-14728422 ] Stefania commented on CASSANDRA-9304: - The {{RateLimiter}} still seems a bit off. It looked kind of wrong before as you pointed out. It's not terribly important but I think this line {{self.current_rate = (self.current_rate + new_rate) / 2.0}} was meant as an average between the current rate and the new one. So the first time, when {{current_rate}} is zero, it should not divide by 2 or else we report half the rate. Secondly, when we calculate the new rate as {{n / difference}}, we may miss records because {{n}} is the number of records passed to every call whilst {{difference}} is the time elapsed since the last time we logged. I wouldn't calculate the rate every time either, but only when logging it. If {{current_record}} cannot be reset to zero after logging it (maybe this was the initial intention of the existing code), then we need a new counter which gives the number of records accumulated between each log point. It's great we now test for all partitioners but we are only exporting 1 record in {{test_all_datatypes_round_trip}} so a better candidate would have been {{test_round_trip}}, where at least we export 10K records. So would you mind adapting {{test_round_trip}} to also run with every partitioner? In fact it would be good to have a bulk round-trip test as well (only for the default partitioner) where we export and import 1M records? We would need to use cassandra stress to write the records. Then we just check the counts. This is just a suggestion. I had problems when running the cqlsh_tests locally: {code} nosetests -s cqlsh_tests {code} {code} == ERROR: test_source_glass (cqlsh_tests.cqlsh_tests.TestCqlsh) -- Traceback (most recent call last): File "/home/stefania/git/cstar/cassandra-dtest/tools.py", line 252, in wrapped f(obj) File "/home/stefania/git/cstar/cassandra-dtest/cqlsh_tests/cqlsh_tests.py", line 341, in test_source_glass self.verify_glass(node1) File "/home/stefania/git/cstar/cassandra-dtest/cqlsh_tests/cqlsh_tests.py", line 102, in verify_glass 'I can eat glass and it does not hurt me': 'Is' File "/home/stefania/git/cstar/cassandra-dtest/cqlsh_tests/cqlsh_tests.py", line 95, in verify_varcharmap got = {k.encode("utf-8"): v for k, v in rows[0][0].iteritems()} IndexError: list index out of range >> begin captured logging << dtest: DEBUG: cluster ccm directory: /tmp/dtest-Ldxvcq - >> end captured logging << - == FAIL: test_all_datatypes_read (cqlsh_tests.cqlsh_copy_tests.CqlshCopyTest) -- Traceback (most recent call last): File "/home/stefania/git/cstar/cassandra-dtest/cqlsh_tests/cqlsh_copy_tests.py", line 690, in test_all_datatypes_read self.assertCsvResultEqual(self.tempfile.name, results) File "/home/stefania/git/cstar/cassandra-dtest/cqlsh_tests/cqlsh_copy_tests.py", line 153, in assertCsvResultEqual raise e AssertionError: Element counts were not equal: First has 1, Second has 0: ['ascii', '1099511627776', '0xbeef', 'True', '3.140124344978758017532527446746826171875', '2.444', '1.1', '127.0.0.1', '25', '\xe3\x83\xbd(\xc2\xb4\xe3\x83\xbc\xef\xbd\x80)\xe3\x83\x8e', '2005-07-14 12:30:00', '2b4e32ce-51de-11e5-85b7-0050b67e8b2f', '830bc4cd-a790-4ac2-85f9-648b0a71306b', 'asdf', '36893488147419103232'] First has 0, Second has 1: ['ascii', '1099511627776', '0xbeef', 'True', '3.140124344978758017532527446746826171875', '2.444', '1.1', '127.0.0.1', '25', '\xe3\x83\xbd(\xc2\xb4\xe3\x83\xbc\xef\xbd\x80)\xe3\x83\x8e', '2005-07-14 04:30:00', '2b4e32ce-51de-11e5-85b7-0050b67e8b2f', '830bc4cd-a790-4ac2-85f9-648b0a71306b', 'asdf', '36893488147419103232'] >> begin captured logging << dtest: DEBUG: cluster ccm directory: /tmp/dtest-cSohP9 dtest: DEBUG: Importing from csv file: /tmp/tmpJgdPJc dtest: WARNING: Mismatch at index: 10 dtest: WARNING: Value in csv: 2005-07-14 12:30:00 dtest: WARNING: Value in result: 2005-07-14 04:30:00 - >> end captured logging << - -- Ran 69 tests in 1161.775s FAILED (SKIP=5, errors=1, failures=1) {code} I scheduled new CI jobs on my view: http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-9304-testall/ http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-9304-dtest/ Let's see if they too report the problems I had locally. > COPY TO
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14995943#comment-14995943 ] Stefania commented on CASSANDRA-9304: - CI is clean on Linux and Windows 2.2. I did not run 3.0 on Windows since the job takes a long time and the patch applied cleanly. I also did not re-run the 2.2 Windows job after the latest rebase but all 3 Linux jobs had no problems with the rebase. I believe the last 6 commits still need review, cc [~thobbs]; aside from this I have nothing else planned for this patch. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004557#comment-15004557 ] Tyler Hobbs commented on CASSANDRA-9304: Great, thanks for taking care of the windows problems! Here are some review comments on the current patch: In {{write_rows_to_csv()}}, instead of using {{sys.exec_info()\[0\]}}, it seems like we could just use the caught exception (by doing {{except Exception, e:}}). Am I missing a reason why that won't work here? In {{get_ranges()}}, this line will result in two token ranges using the same dict, so the count of attempts and rows can become incorrect: {code} ranges[(previous, None)] = ranges[(previous_previous, previous)] {code} Just using {{.copy()}} on the dict should be fine. In {{report_error()}}, the {{issubclass()}} check should be {{isinstance()}} instead. If a child process dies, we don't (and can't) make any adjustments to the {{suceeded}}/{{failed}} count. Because the jobs that were queued to that process aren't resubmitted, this results in the copy process looping endlessly. I see two possible solutions: # Track which token ranges we've submitted to each process. When a child process dies, re-submit any of those token ranges for which we have 0 rows. # Error the entire copy process I'm personally okay with option #2, so feel free to go with that if you don't want to do the work for #1. I was able to find these bugs by editing the code to some simple fault injection in {{start_job()}}: {code} if random.random() > 0.7: future = session.execute_async("SELECT * FROM badtable") elif random.random() > 0.99: sys.exit(1) else: future = session.execute_async(query) {code} I think it would be a good idea to expose something like this for testing through an environment variable. That would allow us to easily exercise these different error scenarios in dtests that are otherwise hard to replicate. Overall, though, I'm a big fan of this patch. You've done excellent work on this so far. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x, 2.2.x, 3.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15006279#comment-15006279 ] Stefania commented on CASSANDRA-9304: - Thank you for the review. Here is the latest update: bq. In {{write_rows_to_csv()}}, instead of using {{sys.exec_info()\[0\]}}, it seems like we could just use the caught exception (by doing {{except Exception, e:}}). Am I missing a reason why that won't work here? No reason, I changed it, thanks. {quote} In {{get_ranges()}}, this line will result in two token ranges using the same dict, so the count of attempts and rows can become incorrect: {code} ranges[(previous, None)] = {code} Just using {{.copy()}} on the dict should be fine. {quote} Fixed, thank you. bq. In {{report_error()}}, the {{issubclass()}} check should be {{isinstance()}} instead. Changed, thanks. {quote} If a child process dies, we don't (and can't) make any adjustments to the suceeded/failed count. Because the jobs that were queued to that process aren't resubmitted, this results in the copy process looping endlessly. I see two possible solutions: # Track which token ranges we've submitted to each process. When a child process dies, re-submit any of those token ranges for which we have 0 rows. # Error the entire copy process I'm personally okay with option #2, so feel free to go with that if you don't want to do the work for #1. {quote} It's not easy to track which process is handling a range because we are using a shared queue where each process takes the next job whenever it is ready to accept another job. So I've opted for option 2 since if a process dies chances are the other processes will die too. {quote} I was able to find these bugs by editing the code to some simple fault injection in start_job(): {code} if random.random() > 0.7: future = session.execute_async("SELECT * FROM badtable") elif random.random() > 0.99: sys.exit(1) else: future = session.execute_async(query) {code} I think it would be a good idea to expose something like this for testing through an environment variable. That would allow us to easily exercise these different error scenarios in dtests that are otherwise hard to replicate. {quote} Done. I've also created a [dtest|https://github.com/stef1927/cassandra-dtest/commit/a629352aa4735baeef42aae81c3bb098e45b77db] but due to the random nature of failure injection there isn't much to check other than checking that it doesn't time-out. Do you think this test is useful at all? I've relaunched one more set of CI jobs, they are still running. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x, 2.2.x, 3.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15006396#comment-15006396 ] Stefania commented on CASSANDRA-9304: - There's one test failing in 2.1: http://cassci.datastax.com/job/stef1927-9304-2.1-dtest/25/testReport/cqlsh_tests.cqlsh_copy_tests/CqlshCopyTest/test_all_datatypes_write/ It passes on the 2.2 and 3.0 branches and locally on the 2.1 branch with the 3.0rc driver installed. It seems the set returned by SELECT is not sorted on Jenkins. I've noticed a similar problem on the unpatched branch: http://cassci.datastax.com/job/cassandra-2.1_dtest/lastCompletedBuild/testReport/cqlsh_tests.cqlsh_copy_tests/CqlshCopyTest/test_all_datatypes_write/ Except here it is even worse because we have not added {{SortedSet}} to formatting.py (which I have done for this patch). > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x, 2.2.x, 3.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15009674#comment-15009674 ] Tyler Hobbs commented on CASSANDRA-9304: The code changes in the latest commit look good to me. As far as the failure injection testing goes, you can make it more deterministic -- I was just using {{random()}} for exploratory testing. For example, have one environment variable that specifies a number of query failures for a single token range. Then, if the token range you're handling includes a particular token, fail the query the specified number of times. This would allow us to test both the case where the failures are less than {{max_attempts}} and the case where they are more than {{max_attempts}}. You could use a similar approach with a second environment variable for child process failure. I know this will make the code a touch uglier, but I think it's worth it for the improved testability. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x, 2.2.x, 3.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15010185#comment-15010185 ] Stefania commented on CASSANDRA-9304: - I've implemented deterministic failure injection pretty much as you suggested except I've encoded a dictionary as json in an environment variable rather than using multiple environment variables, see the [latest commit|https://github.com/stef1927/cassandra/commit/9a8098668498c5959d588bc867495331aa05f6a6] on the 2.1 branch. I've also recreated the 2.2+ branches with a clean merge all the way up to trunk. The proposed dtests are in the [latest commit|https://github.com/stef1927/cassandra-dtest/commit/ffbebd88cfba16759f2f61e3a0790625ca12feaf] of my [dtest branch|https://github.com/stef1927/cassandra-dtest/commits/9304]. I'll wait for your OK before creating a pull request. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x, 2.2.x, 3.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15010331#comment-15010331 ] Stefania commented on CASSANDRA-9304: - CI has completed. We still have the [same problem|http://cassci.datastax.com/job/stef1927-9304-2.1-dtest/26/testReport/cqlsh_tests.cqlsh_copy_tests/CqlshCopyTest/test_all_datatypes_write/] with sorted sets on 2.1 which I tried to fix with [this commit|https://github.com/stef1927/cassandra-dtest/commit/7256a3ddcfe37cd2a1585a37e2527f8e265c8b44], not yet tested since I cannot reproduce it locally. The remaining failures on 3.0 are unrelated connect timeout problems. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x, 2.2.x, 3.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15012446#comment-15012446 ] Stefania commented on CASSANDRA-9304: - dtest pull request: https://github.com/riptano/cassandra-dtest/pull/674 > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x, 2.2.x, 3.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15021396#comment-15021396 ] Stefania commented on CASSANDRA-9304: - [~thobbs]: Here are the [2.1|https://github.com/stef1927/cassandra/commits/9304-b-2.1] and [2.2|https://github.com/stef1927/cassandra/commits/9304-b-2.2] patches. I've also opened CASSANDRA-10753 to fix other unrelated cqlshlib tests; I've asked [~pauloricardomg] to review it but feel free to take that ticket too. I did not start CI since it is only a change in import syntax, you can refer to the CI for CASSANDRA-10753, since its patch has the import syntax also fixed. As usual the 2.2 patch then up-merges without conflicts. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 2.1.12, 2.2.4, 3.0.1, 3.1 > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15021682#comment-15021682 ] Stefania commented on CASSANDRA-9304: - It looks like we also have a couple of unstable tests on Jenkins: http://cassci.datastax.com/job/cassandra-3.0_dtest/lastCompletedBuild/testReport/cqlsh_tests.cqlsh_copy_tests/CqlshCopyTest/test_copy_to_with_more_failures_than_max_attempts/ http://cassci.datastax.com/job/cassandra-2.2_dtest/lastCompletedBuild/testReport/cqlsh_tests.cqlsh_copy_tests/CqlshCopyTest/test_copy_to_with_child_process_crashing/ Unfortunately I cannot reproduce it locally but I am running tests on Jenkins using a custom dtest branch to try and find a fix. As a first thing I've tried increasing the range of failing queries. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 2.1.12, 2.2.4, 3.0.1, 3.1 > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15023723#comment-15023723 ] Stefania commented on CASSANDRA-9304: - This should be fixed by this [pull request|https://github.com/riptano/cassandra-dtest/pull/682]. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 2.1.12, 2.2.4, 3.0.1, 3.1 > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14646980#comment-14646980 ] David Kua commented on CASSANDRA-9304: -- [~Stefania] thank you! I've updated the 9304 branch to resolve most of the points you wrote. I still need to test capping the number of processes at 4. 12 jobs was just the number of jobs that could be chained at once before the cluster would fail for me. However I was testing within a vagrant box and will be testing on my base machine soon. I couldn't think of a better dynamic number :/ > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: David Kua >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14650192#comment-14650192 ] Stefania commented on CASSANDRA-9304: - Thanks, looking good so far. * We should be careful about the number of jobs actually, we should perhaps use a conservative number and let people increase it via a configuration parameter? * In {{RateMeter.increment}} I don't understand why the {{elif (self.current_record % self.log_rate == 0):}} has been replaced with a simple {{else}}, doesn't this change the behavior? Do you have a dtest branch where you are adding the extended tests? > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: David Kua >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14908749#comment-14908749 ] Stefania commented on CASSANDRA-9304: - Resuming progress with this since the patch is almost complete and to avoid unnecessary conflicts I prefer to base CASSANDRA-9302 and CASSANDRA-9303 on this patch. Could we assign a new reviewer please? > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14940741#comment-14940741 ] Jonathan Ellis commented on CASSANDRA-9304: --- [~thobbs] or [~iamaleksey] will you have time to review? > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14940900#comment-14940900 ] Stefania commented on CASSANDRA-9304: - The 2.1 patch is ready for review, I will create the 2.2+ patches once review is completed. This is the first of 3 tickets, the other two are CASSANDRA-9302 and CASSANDRA-9303, where we will continue the refactoring initiated in this ticket for COPY TO and extend it to COPY FROM and the COPY options. As a final step the new classes will be moved to a new file (once COPY FROM is also enhanced). It would help if the person reviewing this ticket could also review the other two tickets once they are available. 2.1 CI: http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-9304-2.1-dtest/ > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14941263#comment-14941263 ] Tyler Hobbs commented on CASSANDRA-9304: Unless [~iamaleksey] is really eager to take this, I can review it next week. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14941966#comment-14941966 ] Stefania commented on CASSANDRA-9304: - Thanks [~thobbs]! > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14943139#comment-14943139 ] Stefania commented on CASSANDRA-9304: - I think I should still add one more thing to the patch: we should put a cap on the maximum number of connections to protect us against very large clusters. I need to address some more urgent tickets before this however, so you can review the existing code anyway and then I will implement this along with the code review comments if that's OK. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14945873#comment-14945873 ] Tyler Hobbs commented on CASSANDRA-9304: Overall the patch is definitely a nice improvement! Most of my review comments are about using more idiomatic python. Imports: * In the imports, StringIO is imported later. Also, to avoid flake8 warnings, add {{# noqa}} to the {{from io import StringIO}} line. ImportExportConfig: * The way that checks for unhandled options are performed is kind of strange. In perform_csv_import() they're checked immediately, but in perform_csv_export() it's deferred to ExportTask. Additionally, a lot of functions seem to modify the options inside an ImportExportConfig instance, which is somewhat unexpected behavior and seems likely to result in bugs in the future. * In Python it's pretty unusual to use a class to only store mutable data, so I would replace the ImportExportConfig class with a function that returns a tuple (e.g. {{(csv_options, dialect_options, unrecognized_options)}}) or a dict. perform_csv_import(): * Why do {{csv_options = config.csv_options}} if it's only used once a couple of lines later? * As a matter of code style, avoid using tuple unpacking unless you're actually unpacking a tuple or the assignments are very trivial (e.g. {{a, b = None, None}}) ExportTask: * The class should inherit from object * get_ranges(): ** There's no need to use a lambda sorting key on a list of tuples. Tuples are naturally sorted sanely. ** Instead of {{del hosts\[:\]}}, just say {{hosts = \[\]}}. You can also remove {{hosts = \[hostname\]}} above. ** Instead of doing {{for entry in ring}} and then using {{entry\[0\]}} and {{entry\[1\]}}, you can unpack the tuple in the loop: {{for token, replicas in ring:}} ** The final {{ranges.append()}} call could use a comment for explanation. If you make the suggested changes to {{hosts}}, just use {{ranges\[-1\]\[1\]}} for the hosts (unless it's empty). This approach is a little more obvious for readers. * In {{send_initial_work()}}, it's idiomatic in python to use {{_}} for the name of an unused variable, so the loop should be: {{for _ in xrange(num_parallel_jobs)}}. ExportProcess: * In {{start_job()}}, it's not clear what would result in an IndexError and why we're ignoring it. Try to limit try/excepts to the minimum number of lines they need to contain, and add a comment explaining what's going on there. * ExportSession should be a separate top-level class instead of a nested one. (It's pretty unusual to nest classes.) Also, it should inherit from object(). * I would rename {{handle_result}} to {{attach_callbacks}} and rename {{return_result}} to {{write_rows_to_csv}}. * The host selection code in {{get_session}} is over-complicated. How about something like: {code} new_hosts = [h for h in hosts if h not in self.hosts_to_sessions] if new_hosts: host = new_hosts[0] # open Cluster, etc else: host = min(hosts, key=lambda h: self.hosts_to_sessions[h].jobs) session = self.hosts_to_sessions[host] session.jobs += 1 return session {code} General: * Try to add a few more code comments explaining what's going on at a high level. Documenting method return values is also good (since there's no type information about that). * You may want to put the csv-related classes and functions into separate modules under {{pylib}}. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14954207#comment-14954207 ] Stefania commented on CASSANDRA-9304: - Thanks for your feedback [~thobbs] and sorry for the delay in addressing it. bq. In the imports, StringIO is imported later. Also, to avoid flake8 warnings, add # noqa to the from io import StringIO line. I removed {{StringIO.StringIO}}, I believe {{io.StringIO}} is equivalent and preferred to {{StringIO.StringIO}}? If so, {{# noqa}} doesn't seem to be required, the first {{E402}} is at line 117. bq. The way that checks for unhandled options are performed is kind of strange. In perform_csv_import() they're checked immediately, but in perform_csv_export() it's deferred to ExportTask. Additionally, a lot of functions seem to modify the options inside an ImportExportConfig instance, which is somewhat unexpected behavior and seems likely to result in bugs in the future. I was planning to add an {{ImportTask}} in CASSANDRA-9302 and to focus more on options in CASSANDRA-9303 but I've patched it up so that it is in a better shape. Let me know if I missed anything. bq. In Python it's pretty unusual to use a class to only store mutable data, so I would replace the ImportExportConfig class with a function that returns a tuple (e.g. (csv_options, dialect_options, unrecognized_options)) or a dict. Replaced with a function that returns {{csv_options, dialect_options, unrecognized_options}} bq. Why do csv_options = config.csv_options if it's only used once a couple of lines later? No longer applicable. bq. As a matter of code style, avoid using tuple unpacking unless you're actually unpacking a tuple or the assignments are very trivial (e.g. a, b = None, None) I cannot find the line you are referring to. bq. ExportTask should inherit from object Done {quote} get_ranges(): There's no need to use a lambda sorting key on a list of tuples. Tuples are naturally sorted sanely. Instead of del hosts\[:\], just say hosts = \[\]. You can also remove hosts = \[hostname\] above. Instead of doing for entry in ring and then using entry\[0\] and entry\[1\], you can unpack the tuple in the loop: for token, replicas in ring: The final ranges.append() call could use a comment for explanation. If you make the suggested changes to hosts, just use ranges\[-1\]\[1\] for the hosts (unless it's empty). This approach is a little more obvious for readers. {quote} Done bq. In send_initial_work(), it's idiomatic in python to use _ for the name of an unused variable, so the loop should be: for _ in xrange(num_parallel_jobs). Done bq. In start_job(), it's not clear what would result in an IndexError and why we're ignoring it. Try to limit try/excepts to the minimum number of lines they need to contain, and add a comment explaining what's going on there. Leftover from the initial patch, it's gone now. bq. ExportSession should be a separate top-level class instead of a nested one. (It's pretty unusual to nest classes.) Also, it should inherit from object(). Done bq. I would rename handle_result to attach_callbacks and rename return_result to write_rows_to_csv. Done {quote} The host selection code in get_session is over-complicated. How about something like: {code} new_hosts = [h for h in hosts if h not in self.hosts_to_sessions] if new_hosts: host = new_hosts[0] # open Cluster, etc else: host = min(hosts, key=lambda h: self.hosts_to_sessions[h].jobs) session = self.hosts_to_sessions[host] session.jobs += 1 return session {code}{quote} Done, thank you. Do you think we need anything else to support very large clusters? For example should we close connections sooner rather than at the very end? bq. Try to add a few more code comments explaining what's going on at a high level. Documenting method return values is also good (since there's no type information about that). Done. bq. You may want to put the csv-related classes and functions into separate modules under pylib. I'll do this after you've had a chance to check all other points above. I also had to fix a problem introduced by an upgrade to the python driver in order to run the round-trip tests, the same fix will be delivered by CASSANDRA-10507. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- Thi
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14955515#comment-14955515 ] Tyler Hobbs commented on CASSANDRA-9304: The new code comments are very nice, thank you for putting those in. The rest of the changes look pretty good to me as well. bq. I removed StringIO.StringIO, I believe io.StringIO is equivalent and preferred to StringIO.StringIO? If so, # noqa doesn't seem to be required, the first E402 is at line 117. It looks like my version of flake8 was old and didn't handle {{try/except}} style imports well. Ignore that comment :) However, there are a couple of other legit flake8 warnings: {noformat} bin/cqlsh|2216| W806 local variable 'ks' is assigned to but never used bin/cqlsh|2217| W806 local variable 'cf' is assigned to but never used {noformat} I also tested this out with a 1m row insert from stress and was surprised to see that I got some timeouts on multiple ranges. These were {{OperationTimedOut}} errors, so it's not immediately clear where the hangup is. I did notice that the current error handling code loses the original exception class (which can be useful), so I suggest changing {{err_callback()}} from: {code} self.pipe.send((token_range, Exception(err.message))) {code} to {code} self.pipe.send((token_range, Exception(err.__class__.__name__ + " - " + err.message))) {code} To avoid the timeouts, I experimented with lowering the page size from 5k to 1k. This did resolve the timeouts for me, and also smoothed the throughput. I suggest that we lower the page size (by doing {{session.default_fetch_size = N}}) to 1k just to lower the impact on nodes. Additionally, we probably want to add some basic timeout recovery. The {{err_callback()}} could perform exponential backoff for a limited number of attempts if an {{OperationTimedOut}} is thrown. To handle coordinator-level timeouts, we could subclass {{cassandra.policies.RetryPolicy}} with an {{on_read_timeout()}} that performs exponential backoff for a limited number of attempts. You can pass an instance of this to the Cluster constructor: {{Cluster(..., retry_policy=foo)}}. Sorry for the additional work, I just don't want to end up with a {{COPY TO}} that goes fast enough to hit timeouts without any sort of recourse for users. That's something that we already have a bit of a problem with for {{COPY FROM}}. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14964855#comment-14964855 ] Stefania commented on CASSANDRA-9304: - [~thobbs] I've resumed this today, sorry for the delay. To retry in {{err_callback()}} when we get an {{OperationTimedOut}}, we either have to create a fresh new query being careful to avoid passing data for the pages that we already retrieved, or use the protected methods and properties of {{ReponseFuture}} to retry the current page, which is more efficient. Do you have any preferences? We should perhaps go with the second choice and log a driver enhancement request? Ideally the retry policy should cover {{OperationTimedOut}} as well, so we only have one path of execution? > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14966519#comment-14966519 ] Stefania commented on CASSANDRA-9304: - The latest changes are ready for review. In the end I've decided to implement exponential back-off only for server side timeouts, that is only in the retry policy. For driver timeouts, {{OperationTimedOut}}, it is problematic to retry because not only do we need to keep track of how many pages we've already received, but we may also retrieve more data from the server. This results in duplicated data. So what I did instead is to increase the timeout with the page size (10 seconds per 1000 entries in the page size at the moment but maybe this is a bit too much). This should eliminate driver side timeouts that result in more data being received from the server. {{OperationTimedOut}}, if still received, would then signal a real connection problem. In this case, it is the parent process that may resubmit the same token range later on, up to a maximum number of times and provided that we have received no data yet. This is true for any errors reported for a range by a worker process. If we have already received data for that range, I decided against retrying to avoid duplication of data. I hope this makes sense, let me know if you do have other preferences on how to implement the back-off and retry mechanism. I've also done the following: * enhanced debug messages and error logging * fixed COPY command completions * added monitoring of child processes in case they die without sending the termination flag on the pipe * fixed possible concurrent access to {{ExportSession.jobs}} Still to do: * Moving the code to a separate file * Testing on Windows > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14968404#comment-14968404 ] Stefania commented on CASSANDRA-9304: - I pushed one more commit to fix a [broken dtest|http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-9304-2.1-dtest/lastCompletedBuild/testReport/cqlsh_tests.cqlsh_tests/TestCqlsh/test_source_glass/]. It seems {{cStringIO}} does not support Unicode strings that cannot be encoded as ascii. I've reintroduced {{StringIO.StringIO}}; performance does not seem to be affected. I've also changed string formatting and removed {{OrderedDict}} in order to continue supporting python 2.6, see discussion on CASSANDRA-10415. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14968782#comment-14968782 ] Stefania commented on CASSANDRA-9304: - A really interesting fact, it takes about 55 seconds to export 500K rows on my box, with a single node running. If instead of formatting rows we just dump binary rows, then it takes 5 seconds total. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14970961#comment-14970961 ] Brian Hess commented on CASSANDRA-9304: [~Stefania] have you re-run the benchmark that [~dkua] performed above (or something similar) - specifically, comparing the "old version" to this new version (and to the cassandra-unloader (https://github.com/brianmhess/cassandra-loader))? What performance improvement do we see? > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14971400#comment-14971400 ] Tyler Hobbs commented on CASSANDRA-9304: Interesting. The current formatting call stack is very deep and expensive, which is okay for normal cqlsh usage, but could definitely be the bottleneck for {{COPY TO}}. I bet if we pre-fetched the formatting functions for each expected type and stored them in a local list, we could improve those numbers dramatically. I can open another ticket for that when I review your recent changes later today. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14973855#comment-14973855 ] Stefania commented on CASSANDRA-9304: - Actually it was bugging me too much and so I fixed it in [this commit|https://github.com/stef1927/cassandra/commit/351901d65d4bcc9c03277a14a804cad996bb53a7]. Pre-fetching the formatters helped a bit but by far the biggest culprit was this: {code} @formatter_for('bytearray') def format_value_blob(val, colormap, **_): -bval = '0x' + ''.join('%02x' % c for c in val) +bval = '0x' + binascii.hexlify(val) return colorme(bval, colormap, 'blob') formatter_for('buffer')(format_value_blob) {code} Disabling coloring also helps (50 seconds vs 35 seconds for 2M records). This part is a bit of a hack, let me know if you prefer to pass a new parameter to all formatters. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14973861#comment-14973861 ] Stefania commented on CASSANDRA-9304: - Here are the results for 2M records, stress generated, with only 1 node running locally on my box (i7-4600U CPU @ 2.10GHz quad-core, 7652MB, SSD): |cassandra-unloader|12 seconds| |9304-2.1 branch|35 seconds| |dkua/9304|2 minutes, 53 seconds| |cassandra-2.2 branch|6 minutes, 28 seconds| Notes: * on the cassandra-2.1 branch, COPY TO is currently broken, that's why I used the 2.2 branch * the results of the latest 9304-2.1 vary according to number of threads and page size selected, not sure if we can do still better (without fixing the byte array formatting bug discussed above, we were at 3 minutes 45 seconds, worse than the original implementation which had a simpler but perhaps more effective job scheduling policy) > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14979489#comment-14979489 ] Tyler Hobbs commented on CASSANDRA-9304: [~Stefania] very nice work! You did a very good job with the error handling and backoff. You're also getting pretty good at python :) I think the rate of ~60k rows/sec on a laptop is good for now. We could probably double that, but I think most of the low-hanging fruit has been picked here. Your latest commits look good to me, and I'm fine with the {{NO_COLOR_MAP}} fix. Regarding Windows testing, maybe [~JoshuaMcKenzie] can weight in on who can help with that? > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14979570#comment-14979570 ] Stefania commented on CASSANDRA-9304: - Thanks [~thobbs]! bq. Regarding Windows testing, maybe Joshua McKenzie can weight in on who can help with that? I can test on Windows myself; I've postponed setting up dtests on Windows for long enough. :) > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14988524#comment-14988524 ] Stefania commented on CASSANDRA-9304: - Unfortunately testing on Windows revealed there is a bit more work to do. It appears multiprocessing on Windows is more limited: because of the lack of {{os.fork}}, the worker process class has to be _picklable_. This means changing a few things. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14989270#comment-14989270 ] Stefania commented on CASSANDRA-9304: - I finally got the pickle part to work but there is still something strange going on for version 2.1 on Windows. I am receiving {{str}} objects rather than {{bytearray}} objects for blob db values. I've debugged it all the way down to _protocol.py_ in the driver, in {{recv_results_rows}}. When called from the main process, {{ctype.from_binary}} returns a {{bytearray}} as expected but when called from a sub-process it returns the same string passed in as an argument. It seems in this second case it is {{_CassandraType.deserialize}} that gets called whilst in the case that works there is another method that gets called but I cannot find it. [~aholmber], do you have any pointers? The driver has been built without compilation ({{--no-extensions}} and I've added the {{cassandra}} source folder directly to the system path for easier debugging but the same problem occurs with the cqlsh embedded driver. [~thobbs], I cannot get multiprocessing to work for 2.1 unless I rename _cqlsh_ to _cqlsh.py_ like it has been done for 2.2. I don't think there is much we can do about this. Shall I rename the file in 2.1 or shall we accept that COPY on Windows will only work from 2.2 onward? Finally, I also have an issue with pipes, in that poll or select don't work. I can work around the lack of select by polling individual pipes, it is clearly stated in the documentation that Windows supports select only for sockets, but I still don't know why polling individual pipes doesn't work either, at least not reliably. I haven't spent much time on this but it looks like it needs some sort of flushing when sending. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14989632#comment-14989632 ] Philip Thompson commented on CASSANDRA-9304: I expect it's acceptable for COPY on Windows to only work from 2.2 onward, given 2.2 is when Windows support became official, and not just beta. [~JoshuaMcKenzie], thoughts? > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14989639#comment-14989639 ] T Jake Luciani commented on CASSANDRA-9304: --- bq. I expect it's acceptable for COPY on Windows to only work from 2.2 onward Agreed. [~Stefania] I had to do this for CASSANDRA-9795 > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14989753#comment-14989753 ] Joshua McKenzie commented on CASSANDRA-9304: +1 to 2.2+ on Windows. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14990243#comment-14990243 ] Adam Holmberg commented on CASSANDRA-9304: -- [~Stefania] nothing jumps out at me after a brief scan of the changeset. First thing I would check is the [column metadata materialized for this column|https://github.com/datastax/python-driver/blob/2.7.2/cassandra/protocol.py#L625] (is it the expected type?). I have not tried to reproduce yet. Does the subsequent discussion about Windows support for 2.2+ make this "don't care", or is there still an issue with other versions? > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14991279#comment-14991279 ] Stefania commented on CASSANDRA-9304: - Thank you for your input. Regarding version support for Windows, fine for 2.2+ but for completeness I'll point out that the only obstacle left in 2.1 is the name of the file (_cqlsh_ -> _cqlsh.py_). Regarding the problem with pipes, I've replaced pipes with queues so we don't need to deal with the low level platform specific details. Queues can also be safely used from the callback threads, which was not the case for pipes. Regarding the problem with the driver, I haven't tested in 2.2 but I don't think it matters which version since yesterday I was using the latest cassandra-test driver version. Today I used 2.7.2. The column type is the same, {{cassandra.cqltypes.BytesType}}, the method called from {{recv_result_rows()}} is the same, {{>}} but {{cls.serialize}} in {{from_binary}} is a lambda for the case that works and the default implementation {{CassandraType.deserialize}} for the case that does not work. I don't know where the lambda comes from but I noticed there is a cython deserialize for {{BytesType}} in deserializers.pyx. I don't know how cython works but if this is picked up in the normal case then the problem is again with the way multiprocessing imports modules. The problem can be solved by adding a deserialize implementation to BytesType, like it's done for other types: {code} Stefi@Lila MINGW64 ~/git/cstar/python-driver ((2.7.2)) $ git diff diff --git a/cassandra/cqltypes.py b/cassandra/cqltypes.py index f39d28b..eb8d3b6 100644 --- a/cassandra/cqltypes.py +++ b/cassandra/cqltypes.py @@ -350,6 +350,10 @@ class BytesType(_CassandraType): def serialize(val, protocol_version): return six.binary_type(val) +@staticmethod +def deserialize(byts, protocol_version): +return bytearray(byts) + class DecimalType(_CassandraType): typename = 'decimal' {code} If this is not enough and you want to debug some more [~aholmber], you can use the 2.1 patch attached. I'm still working on the 2.2. merge. You need to generate a table with a blob, I used cassandra-stress. Then run {{COPY TO 'anyfile';}} from cqlsh and this should result in a Unicode decode error on Windows because the blob is received as a string. If you prefer me to test things for you, that works too. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14991364#comment-14991364 ] Stefania commented on CASSANDRA-9304: - The [2.2 patch|https://github.com/stef1927/cassandra/tree/9304-2.2] is also available now. It applies without conflict to 3.0. CI: http://cassci.datastax.com/job/stef1927-9304-2.1-dtest/ http://cassci.datastax.com/job/stef1927-9304-2.2-dtest/ http://cassci.datastax.com/job/stef1927-9304-3.0-dtest/ http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-9304-2.2-windows-dtest_win32/ > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14991849#comment-14991849 ] Adam Holmberg commented on CASSANDRA-9304: -- bq. If this is not enough and you want to debug some more ... you can use the 2.1 patch attached I do think we should debug more. bq. I am receiving str objects rather than bytearray objects for blob db values. I'm sorry I didn't key in on this earlier, but the driver never returns bytearray for blob types. It returns the binary type of the runtime ({{str}} for Python 2, {{bytes}} for Python 3). I don't think this should change. I suspect that you're losing this patch when the child processes are created: https://github.com/apache/cassandra/blob/f62d6e407d859c0c8da5283902a5153e4bd67b5c/bin/cqlsh#L678-L680 I'm sure you're aware, but the {{multiprocessing}} module has some [Windows-related idiosyncrasies|https://docs.python.org/2/library/multiprocessing.html?highlight=windows]. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14992778#comment-14992778 ] Stefania commented on CASSANDRA-9304: - bq. I suspect that you're losing this patch when the child processes are created This explains it perfectly, thank you! I can't believe I missed it, I did a wide search for {{BytesType}} in the driver, never thinking cqlsh would change its behavior. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Minor > Labels: cqlsh > Fix For: 3.x, 2.1.x, 2.2.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14638376#comment-14638376 ] Stefania commented on CASSANDRA-9304: - Really an excellent patch! We need to extend the tests a bit however. {{test_copy_to}} exports a relatively small table with a single node in the cluster. I would add more tests in cqlsh_copy_tests.py. I would test slightly bigger tables using stress to control the table population. I would test with a different number of hosts and use all available partitioners. Another parameter we could change is the number of processes involved in the export but it would require passing this number to cqlsh via config. The code review comments are here: * Ln 58 {{import multiprocessing as multp}}: I personally find {{multp}} not very readable and so I would just avoid the alias but up to you. * Ln 824 {{get_ring(self, ksname)}}: the modification doesn't seem to be required by this patch * Ln 830 {{get_min_token(self)}}: OrderPreservingPartitioner is missing * Ln 839 {{get_token_ranges(self)}}: if ring[] is empty there is an uncaught IndexError at line 848 * Ln 1906: perform_csv_import caps the number of processes at 4, should we cap the number of processes as well? Have you tested with different number of processes for correctness and performance? * Ln 2039: why exactly 12 jobs, add a comment to explain why this number * Ln 2453 {{increment(self, n=1)}}: it could use a bit of refactoring, like introducing a new method for logging, which accepts the rate as a parameter * Ln 1442, 1602, 1870: trailing spaces CI will be visible on my Jenkins view: http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-9304-testall/ http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-9304-dtest/ I also verified that the cqlsh dtests pass locally on my box. > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: David Kua >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9304) COPY TO improvements
[ https://issues.apache.org/jira/browse/CASSANDRA-9304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14638443#comment-14638443 ] Stefania commented on CASSANDRA-9304: - It would also help if you rebased, some of the dtests are failing because of missing code fixes on your branch (e.g. bootstrap_tests). > COPY TO improvements > > > Key: CASSANDRA-9304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9304 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jonathan Ellis >Assignee: David Kua >Priority: Minor > Labels: cqlsh > Fix For: 2.1.x > > > COPY FROM has gotten a lot of love. COPY TO not so much. One obvious > improvement could be to parallelize reading and writing (write one page of > data while fetching the next). -- This message was sent by Atlassian JIRA (v6.3.4#6332)