[ 
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)

Reply via email to