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