You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2016/03/08 10:26:40 UTC

[01/23] cassandra git commit: COPY FROM on large datasets: fix progress report and debug performance

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 0129f70c4 -> c3d2f26f4
  refs/heads/cassandra-2.2 6e0395e5a -> b74ffeafd
  refs/heads/cassandra-3.0 34b07a74f -> 49c616cf0
  refs/heads/cassandra-3.5 c9e9b62eb -> 6329d54a6
  refs/heads/trunk 733d1ee64 -> acde50815


COPY FROM on large datasets: fix progress report and debug performance

patch by Stefania Alborghetti; reviewed by Adam Holmberg for CASSANDRA-11053


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c3d2f26f
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c3d2f26f
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c3d2f26f

Branch: refs/heads/cassandra-2.1
Commit: c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc
Parents: 0129f70
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Thu Jan 28 14:31:55 2016 +0800
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Mar 8 10:19:13 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                |    1 +
 bin/cqlsh                  |   28 +-
 pylib/cqlshlib/copyutil.py | 1160 +++++++++++++++++++++++++--------------
 pylib/cqlshlib/util.py     |   35 +-
 pylib/setup.py             |    2 +
 5 files changed, 809 insertions(+), 417 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index eed9035..d6b085c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.14
+ * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
  * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
  * Don't remove FailureDetector history on removeEndpoint (CASSANDRA-10371)
  * Only notify if repair status changed (CASSANDRA-11172)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/bin/cqlsh
----------------------------------------------------------------------
diff --git a/bin/cqlsh b/bin/cqlsh
index 7a39636..374e588 100755
--- a/bin/cqlsh
+++ b/bin/cqlsh
@@ -417,7 +417,7 @@ COPY_COMMON_OPTIONS = ['DELIMITER', 'QUOTE', 'ESCAPE', 'HEADER', 'NULL', 'DATETI
                        'MAXATTEMPTS', 'REPORTFREQUENCY', 'DECIMALSEP', 'THOUSANDSSEP', 'BOOLSTYLE',
                        'NUMPROCESSES', 'CONFIGFILE', 'RATEFILE']
 COPY_FROM_OPTIONS = ['CHUNKSIZE', 'INGESTRATE', 'MAXBATCHSIZE', 'MINBATCHSIZE', 'MAXROWS',
-                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE']
+                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE', 'PREPAREDSTATEMENTS']
 COPY_TO_OPTIONS = ['ENCODING', 'PAGESIZE', 'PAGETIMEOUT', 'BEGINTOKEN', 'ENDTOKEN', 'MAXOUTPUTSIZE', 'MAXREQUESTS']
 
 
@@ -533,8 +533,23 @@ def insert_driver_hooks():
 
 def extend_cql_deserialization():
     """
-    The python driver returns BLOBs as string, but we expect them as bytearrays
+    The python driver returns BLOBs as string, but we expect them as bytearrays; therefore we change
+    the implementation of cassandra.cqltypes.BytesType.deserialize.
+
+    The deserializers package exists only when the driver has been compiled with cython extensions and
+    cassandra.deserializers.DesBytesType replaces cassandra.cqltypes.BytesType.deserialize.
+
+    DesBytesTypeByteArray is a fast deserializer that converts blobs into bytearrays but it was
+    only introduced recently (3.1.0). If it is available we use it, otherwise we remove
+    cassandra.deserializers.DesBytesType so that we fall back onto cassandra.cqltypes.BytesType.deserialize
+    just like in the case where no cython extensions are present.
     """
+    if hasattr(cassandra, 'deserializers'):
+        if hasattr(cassandra.deserializers, 'DesBytesTypeByteArray'):
+            cassandra.deserializers.DesBytesType = cassandra.deserializers.DesBytesTypeByteArray
+        else:
+            del cassandra.deserializers.DesBytesType
+
     cassandra.cqltypes.BytesType.deserialize = staticmethod(lambda byts, protocol_version: bytearray(byts))
     cassandra.cqltypes.CassandraType.support_empty_values = True
 
@@ -1534,9 +1549,9 @@ class Shell(cmd.Cmd):
 
         Available COPY FROM options and defaults:
 
-          CHUNKSIZE=1000          - the size of chunks passed to worker processes
+          CHUNKSIZE=5000          - the size of chunks passed to worker processes
           INGESTRATE=100000       - an approximate ingest rate in rows per second
-          MINBATCHSIZE=2          - the minimum size of an import batch
+          MINBATCHSIZE=10         - the minimum size of an import batch
           MAXBATCHSIZE=20         - the maximum size of an import batch
           MAXROWS=-1              - the maximum number of rows, -1 means no maximum
           SKIPROWS=0              - the number of rows to skip
@@ -1545,6 +1560,11 @@ class Shell(cmd.Cmd):
           MAXINSERTERRORS=-1      - the maximum global number of insert errors, -1 means no maximum
           ERRFILE=''              - a file where to store all rows that could not be imported, by default this is
                                     import_ks_table.err where <ks> is your keyspace and <table> is your table name.
+          PREPAREDSTATEMENTS=True - whether to use prepared statements when importing, by default True. Set this to
+                                    False if you don't mind shifting data parsing to the cluster. The cluster will also
+                                    have to compile every batch statement. For large and oversized clusters
+                                    this will result in a faster import but for smaller clusters it may generate
+                                    timeouts.
 
         Available COPY TO options and defaults:
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/pylib/cqlshlib/copyutil.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/copyutil.py b/pylib/cqlshlib/copyutil.py
index f9e4a85..cd03765 100644
--- a/pylib/cqlshlib/copyutil.py
+++ b/pylib/cqlshlib/copyutil.py
@@ -1,3 +1,5 @@
+# cython: profile=True
+
 # Licensed to the Apache Software Foundation (ASF) under one
 # or more contributor license agreements.  See the NOTICE file
 # distributed with this work for additional information
@@ -21,25 +23,29 @@ import json
 import glob
 import multiprocessing as mp
 import os
-import Queue
+import platform
+import random
 import re
 import struct
 import sys
 import time
 import traceback
 
+from bisect import bisect_right
 from calendar import timegm
-from collections import defaultdict, deque, namedtuple
+from collections import defaultdict, namedtuple
 from decimal import Decimal
 from random import randrange
 from StringIO import StringIO
+from select import select
 from threading import Lock
 from uuid import UUID
+from util import profile_on, profile_off
 
 from cassandra.cluster import Cluster
 from cassandra.cqltypes import ReversedType, UserType
-from cassandra.metadata import protect_name, protect_names
-from cassandra.policies import RetryPolicy, WhiteListRoundRobinPolicy, TokenAwarePolicy, DCAwareRoundRobinPolicy
+from cassandra.metadata import protect_name, protect_names, protect_value
+from cassandra.policies import RetryPolicy, WhiteListRoundRobinPolicy, DCAwareRoundRobinPolicy
 from cassandra.query import BatchStatement, BatchType, SimpleStatement, tuple_factory
 from cassandra.util import Date, Time
 
@@ -48,6 +54,10 @@ from displaying import NO_COLOR_MAP
 from formatting import format_value_default, EMPTY, get_formatter
 from sslhandling import ssl_settings
 
+PROFILE_ON = False
+STRACE_ON = False
+IS_LINUX = platform.system() == 'Linux'
+
 CopyOptions = namedtuple('CopyOptions', 'copy dialect unrecognized')
 
 
@@ -59,6 +69,81 @@ def safe_normpath(fname):
     return os.path.normpath(os.path.expanduser(fname)) if fname else fname
 
 
+class OneWayChannel(object):
+    """
+    A one way pipe protected by two process level locks, one for reading and one for writing.
+    """
+    def __init__(self):
+        self.reader, self.writer = mp.Pipe(duplex=False)
+        self.rlock = mp.Lock()
+        self.wlock = mp.Lock()
+
+    def send(self, obj):
+        with self.wlock:
+            self.writer.send(obj)
+
+    def recv(self):
+        with self.rlock:
+            return self.reader.recv()
+
+    def close(self):
+        self.reader.close()
+        self.writer.close()
+
+
+class OneWayChannels(object):
+    """
+    A group of one way channels.
+    """
+    def __init__(self, num_channels):
+        self.channels = [OneWayChannel() for _ in xrange(num_channels)]
+        self._readers = [ch.reader for ch in self.channels]
+        self._rlocks = [ch.rlock for ch in self.channels]
+        self._rlocks_by_readers = dict([(ch.reader, ch.rlock) for ch in self.channels])
+        self.num_channels = num_channels
+
+        self.recv = self.recv_select if IS_LINUX else self.recv_polling
+
+    def recv_select(self, timeout):
+        """
+        Implementation of the recv method for Linux, where select is available. Receive an object from
+        all pipes that are ready for reading without blocking.
+        """
+        readable, _, _ = select(self._readers, [], [], timeout)
+        for r in readable:
+            with self._rlocks_by_readers[r]:
+                try:
+                    yield r.recv()
+                except EOFError:
+                    continue
+
+    def recv_polling(self, timeout):
+        """
+        Implementation of the recv method for platforms where select() is not available for pipes.
+        We poll on all of the readers with a very small timeout. We stop when the timeout specified
+        has been received but we may exceed it since we check all processes during each sweep.
+        """
+        start = time.time()
+        while True:
+            for i, r in enumerate(self._readers):
+                with self._rlocks[i]:
+                    if r.poll(0.000000001):
+                        try:
+                            yield r.recv()
+                        except EOFError:
+                            continue
+
+            if time.time() - start > timeout:
+                break
+
+    def close(self):
+        for ch in self.channels:
+            try:
+                ch.close()
+            except:
+                pass
+
+
 class CopyTask(object):
     """
     A base class for ImportTask and ExportTask
@@ -72,15 +157,18 @@ class CopyTask(object):
         self.protocol_version = protocol_version
         self.config_file = config_file
         # do not display messages when exporting to STDOUT
-        self.printmsg = self._printmsg if self.fname is not None or direction == 'in' else lambda _, eol='\n': None
+        self.printmsg = self._printmsg if self.fname is not None or direction == 'from' else lambda _, eol='\n': None
         self.options = self.parse_options(opts, direction)
 
         self.num_processes = self.options.copy['numprocesses']
+        if direction == 'in':
+            self.num_processes += 1  # add the feeder process
+
         self.printmsg('Using %d child processes' % (self.num_processes,))
 
         self.processes = []
-        self.inmsg = mp.Queue()
-        self.outmsg = mp.Queue()
+        self.inmsg = OneWayChannels(self.num_processes)
+        self.outmsg = OneWayChannels(self.num_processes)
 
         self.columns = CopyTask.get_columns(shell, ks, table, columns)
         self.time_start = time.time()
@@ -166,10 +254,10 @@ class CopyTask(object):
         copy_options['maxattempts'] = int(opts.pop('maxattempts', 5))
         copy_options['dtformats'] = opts.pop('datetimeformat', shell.display_time_format)
         copy_options['float_precision'] = shell.display_float_precision
-        copy_options['chunksize'] = int(opts.pop('chunksize', 1000))
+        copy_options['chunksize'] = int(opts.pop('chunksize', 5000))
         copy_options['ingestrate'] = int(opts.pop('ingestrate', 100000))
         copy_options['maxbatchsize'] = int(opts.pop('maxbatchsize', 20))
-        copy_options['minbatchsize'] = int(opts.pop('minbatchsize', 2))
+        copy_options['minbatchsize'] = int(opts.pop('minbatchsize', 10))
         copy_options['reportfrequency'] = float(opts.pop('reportfrequency', 0.25))
         copy_options['consistencylevel'] = shell.consistency_level
         copy_options['decimalsep'] = opts.pop('decimalsep', '.')
@@ -186,6 +274,7 @@ class CopyTask(object):
         copy_options['errfile'] = safe_normpath(opts.pop('errfile', 'import_%s_%s.err' % (self.ks, self.table,)))
         copy_options['ratefile'] = safe_normpath(opts.pop('ratefile', ''))
         copy_options['maxoutputsize'] = int(opts.pop('maxoutputsize', '-1'))
+        copy_options['preparedstatements'] = bool(opts.pop('preparedstatements', 'true').lower() == 'true')
 
         self.check_options(copy_options)
         return CopyOptions(copy=copy_options, dialect=dialect_options, unrecognized=opts)
@@ -206,14 +295,17 @@ class CopyTask(object):
     def get_num_processes(cap):
         """
         Pick a reasonable number of child processes. We need to leave at
-        least one core for the parent process.  This doesn't necessarily
-        need to be capped, but 4 is currently enough to keep
-        a single local Cassandra node busy so we use this for import, whilst
-        for export we use 16 since we can connect to multiple Cassandra nodes.
-        Eventually this parameter will become an option.
+        least one core for the parent process.
+        """
+        return max(1, min(cap, CopyTask.get_num_cores() - 1))
+
+    @staticmethod
+    def get_num_cores():
+        """
+        Return the number of cores if available.
         """
         try:
-            return max(1, min(cap, mp.cpu_count() - 1))
+            return mp.cpu_count()
         except NotImplementedError:
             return 1
 
@@ -244,28 +336,40 @@ class CopyTask(object):
         return shell.get_column_names(ks, table) if not columns else columns
 
     def close(self):
-        for process in self.processes:
-            process.terminate()
-
+        self.stop_processes()
         self.inmsg.close()
         self.outmsg.close()
 
     def num_live_processes(self):
         return sum(1 for p in self.processes if p.is_alive())
 
+    @staticmethod
+    def get_pid():
+        return os.getpid() if hasattr(os, 'getpid') else None
+
+    @staticmethod
+    def trace_process(pid):
+        if pid and STRACE_ON:
+            os.system("strace -vvvv -c -o strace.{pid}.out -e trace=all -p {pid}&".format(pid=pid))
+
+    def start_processes(self):
+        for i, process in enumerate(self.processes):
+            process.start()
+            self.trace_process(process.pid)
+
+        self.trace_process(self.get_pid())
+
+    def stop_processes(self):
+        for process in self.processes:
+            process.terminate()
+
     def make_params(self):
         """
         Return a dictionary of parameters to be used by the worker processes.
         On Windows this dictionary must be pickle-able.
-
-        inmsg is the message queue flowing from parent to child process, so outmsg from the parent point
-        of view and, vice-versa,  outmsg is the message queue flowing from child to parent, so inmsg
-        from the parent point of view, hence the two are swapped below.
         """
         shell = self.shell
-        return dict(inmsg=self.outmsg,  # see comment above
-                    outmsg=self.inmsg,  # see comment above
-                    ks=self.ks,
+        return dict(ks=self.ks,
                     table=self.table,
                     local_dc=self.local_dc,
                     columns=self.columns,
@@ -281,6 +385,17 @@ class CopyTask(object):
                     debug=shell.debug
                     )
 
+    def update_params(self, params, i):
+        """
+        Add the communication channels to the parameters to be passed to the worker process:
+            inmsg is the message queue flowing from parent to child process, so outmsg from the parent point
+            of view and, vice-versa,  outmsg is the message queue flowing from child to parent, so inmsg
+            from the parent point of view, hence the two are swapped below.
+        """
+        params['inmsg'] = self.outmsg.channels[i]
+        params['outmsg'] = self.inmsg.channels[i]
+        return params
+
 
 class ExportWriter(object):
     """
@@ -414,10 +529,9 @@ class ExportTask(CopyTask):
 
         params = self.make_params()
         for i in xrange(self.num_processes):
-            self.processes.append(ExportProcess(params))
+            self.processes.append(ExportProcess(self.update_params(params, i)))
 
-        for process in self.processes:
-            process.start()
+        self.start_processes()
 
         try:
             self.export_records(ranges)
@@ -468,11 +582,12 @@ class ExportTask(CopyTask):
 
             return ret
 
-        def make_range_data(replicas=[]):
+        def make_range_data(replicas=None):
             hosts = []
-            for r in replicas:
-                if r.is_up and r.datacenter == local_dc:
-                    hosts.append(r.address)
+            if replicas:
+                for r in replicas:
+                    if r.is_up and r.datacenter == local_dc:
+                        hosts.append(r.address)
             if not hosts:
                 hosts.append(hostname)  # fallback to default host if no replicas in current dc
             return {'hosts': tuple(hosts), 'attempts': 0, 'rows': 0}
@@ -542,10 +657,13 @@ class ExportTask(CopyTask):
             return None
 
     def send_work(self, ranges, tokens_to_send):
+        i = 0
         for token_range in tokens_to_send:
-            self.outmsg.put((token_range, ranges[token_range]))
+            self.outmsg.channels[i].send((token_range, ranges[token_range]))
             ranges[token_range]['attempts'] += 1
 
+            i = i + 1 if i < self.num_processes - 1 else 0
+
     def export_records(self, ranges):
         """
         Send records to child processes and monitor them by collecting their results
@@ -568,8 +686,7 @@ class ExportTask(CopyTask):
         succeeded = 0
         failed = 0
         while (failed + succeeded) < total_requests and self.num_live_processes() == num_processes:
-            try:
-                token_range, result = self.inmsg.get(timeout=1.0)
+            for token_range, result in self.inmsg.recv(timeout=0.1):
                 if token_range is None and result is None:  # a request has finished
                     succeeded += 1
                 elif isinstance(result, Exception):  # an error occurred
@@ -594,8 +711,6 @@ class ExportTask(CopyTask):
                     self.writer.write(data, num)
                     meter.increment(n=num)
                     ranges[token_range]['rows'] += num
-            except Queue.Empty:
-                pass
 
         if self.num_live_processes() < len(processes):
             for process in processes:
@@ -612,7 +727,7 @@ class ExportTask(CopyTask):
                        self.describe_interval(time.time() - self.time_start)))
 
 
-class ImportReader(object):
+class FilesReader(object):
     """
     A wrapper around a csv reader to keep track of when we have
     exhausted reading input files. We are passed a comma separated
@@ -620,18 +735,15 @@ class ImportReader(object):
     We generate a source generator and we read each source one
     by one.
     """
-    def __init__(self, task):
-        self.shell = task.shell
-        self.options = task.options
-        self.printmsg = task.printmsg
-        self.chunk_size = self.options.copy['chunksize']
-        self.header = self.options.copy['header']
-        self.max_rows = self.options.copy['maxrows']
-        self.skip_rows = self.options.copy['skiprows']
-        self.sources = self.get_source(task.fname)
+    def __init__(self, fname, options):
+        self.chunk_size = options.copy['chunksize']
+        self.header = options.copy['header']
+        self.max_rows = options.copy['maxrows']
+        self.skip_rows = options.copy['skiprows']
+        self.fname = fname
+        self.sources = None  # must be created later due to pickle problems on Windows
         self.num_sources = 0
         self.current_source = None
-        self.current_reader = None
         self.num_read = 0
 
     def get_source(self, paths):
@@ -640,35 +752,33 @@ class ImportReader(object):
          wrapping the source input, file name and a boolean indicating
          if it requires closing.
         """
-        shell = self.shell
-        LineSource = namedtuple('LineSource', 'input close fname')
-
         def make_source(fname):
             try:
-                ret = LineSource(input=open(fname, 'rb'), close=True, fname=fname)
-                return ret
+                return open(fname, 'rb')
             except IOError, e:
-                shell.printerr("Can't open %r for reading: %s" % (fname, e))
+                self.printmsg("Can't open %r for reading: %s" % (fname, e))
                 return None
 
-        if paths is None:
-            self.printmsg("[Use \. on a line by itself to end input]")
-            yield LineSource(input=shell.use_stdin_reader(prompt='[copy] ', until=r'\.'), close=False, fname='')
-        else:
-            for path in paths.split(','):
-                path = path.strip()
-                if os.path.isfile(path):
-                    yield make_source(path)
-                else:
-                    for f in glob.glob(path):
-                        yield (make_source(f))
+        for path in paths.split(','):
+            path = path.strip()
+            if os.path.isfile(path):
+                yield make_source(path)
+            else:
+                for f in glob.glob(path):
+                    yield (make_source(f))
+
+    @staticmethod
+    def printmsg(msg, eol='\n'):
+        sys.stdout.write(msg + eol)
+        sys.stdout.flush()
 
     def start(self):
+        self.sources = self.get_source(self.fname)
         self.next_source()
 
     @property
     def exhausted(self):
-        return not self.current_reader
+        return not self.current_source
 
     def next_source(self):
         """
@@ -679,40 +789,34 @@ class ImportReader(object):
         while self.current_source is None:
             try:
                 self.current_source = self.sources.next()
-                if self.current_source and self.current_source.fname:
+                if self.current_source:
                     self.num_sources += 1
             except StopIteration:
                 return False
 
         if self.header:
-            self.current_source.input.next()
+            self.current_source.next()
 
-        self.current_reader = csv.reader(self.current_source.input, **self.options.dialect)
         return True
 
     def close_current_source(self):
         if not self.current_source:
             return
 
-        if self.current_source.close:
-            self.current_source.input.close()
-        elif self.shell.tty:
-            print
-
+        self.current_source.close()
         self.current_source = None
-        self.current_reader = None
 
     def close(self):
         self.close_current_source()
 
     def read_rows(self, max_rows):
-        if not self.current_reader:
+        if not self.current_source:
             return []
 
         rows = []
         for i in xrange(min(max_rows, self.chunk_size)):
             try:
-                row = self.current_reader.next()
+                row = self.current_source.next()
                 self.num_read += 1
 
                 if 0 <= self.max_rows < self.num_read:
@@ -729,13 +833,91 @@ class ImportReader(object):
         return filter(None, rows)
 
 
-class ImportErrors(object):
+class PipeReader(object):
     """
-    A small class for managing import errors
+    A class for reading rows received on a pipe, this is used for reading input from STDIN
+    """
+    def __init__(self, inmsg, options):
+        self.inmsg = inmsg
+        self.chunk_size = options.copy['chunksize']
+        self.header = options.copy['header']
+        self.max_rows = options.copy['maxrows']
+        self.skip_rows = options.copy['skiprows']
+        self.num_read = 0
+        self.exhausted = False
+        self.num_sources = 1
+
+    def start(self):
+        pass
+
+    def read_rows(self, max_rows):
+        rows = []
+        for i in xrange(min(max_rows, self.chunk_size)):
+            row = self.inmsg.recv()
+            if row is None:
+                self.exhausted = True
+                break
+
+            self.num_read += 1
+            if 0 <= self.max_rows < self.num_read:
+                self.exhausted = True
+                break  # max rows exceeded
+
+            if self.header or self.num_read < self.skip_rows:
+                self.header = False  # skip header or initial skip_rows rows
+                continue
+
+            rows.append(row)
+
+        return rows
+
+
+class ImportProcessResult(object):
+    """
+    An object sent from ImportProcess instances to the parent import task in order to indicate progress.
+    """
+    def __init__(self, imported=0):
+        self.imported = imported
+
+
+class FeedingProcessResult(object):
+    """
+    An object sent from FeedingProcess instances to the parent import task in order to indicate progress.
+    """
+    def __init__(self, sent, reader):
+        self.sent = sent
+        self.num_sources = reader.num_sources
+        self.skip_rows = reader.skip_rows
+
+
+class ImportTaskError(object):
+    """
+    An object sent from child processes (feeder or workers) to the parent import task to indicate an error.
+    """
+    def __init__(self, name, msg, rows=None, attempts=1, final=True):
+        self.name = name
+        self.msg = msg
+        self.rows = rows if rows else []
+        self.attempts = attempts
+        self.final = final
+
+    def is_parse_error(self):
+        """
+        We treat read and parse errors as unrecoverable and we have different global counters for giving up when
+        a maximum has been reached. We consider value and type errors as parse errors as well since they
+        are typically non recoverable.
+        """
+        name = self.name
+        return name.startswith('ValueError') or name.startswith('TypeError') or \
+            name.startswith('ParseError') or name.startswith('IndexError') or name.startswith('ReadError')
+
+
+class ImportErrorHandler(object):
+    """
+    A class for managing import errors
     """
     def __init__(self, task):
         self.shell = task.shell
-        self.reader = task.reader
         self.options = task.options
         self.printmsg = task.printmsg
         self.max_attempts = self.options.copy['maxattempts']
@@ -771,42 +953,26 @@ class ImportErrors(object):
             for row in rows:
                 writer.writerow(row)
 
-    def handle_error(self, err, batch):
+    def handle_error(self, err):
         """
         Handle an error by printing the appropriate error message and incrementing the correct counter.
-        Return true if we should retry this batch, false if the error is non-recoverable
         """
         shell = self.shell
-        err = str(err)
 
-        if self.is_parse_error(err):
-            self.parse_errors += len(batch['rows'])
-            self.add_failed_rows(batch['rows'])
-            shell.printerr("Failed to import %d rows: %s -  given up without retries"
-                           % (len(batch['rows']), err))
-            return False
+        if err.is_parse_error():
+            self.parse_errors += len(err.rows)
+            self.add_failed_rows(err.rows)
+            shell.printerr("Failed to import %d rows: %s - %s,  given up without retries"
+                           % (len(err.rows), err.name, err.msg))
         else:
-            self.insert_errors += len(batch['rows'])
-            if batch['attempts'] < self.max_attempts:
-                shell.printerr("Failed to import %d rows: %s -  will retry later, attempt %d of %d"
-                               % (len(batch['rows']), err, batch['attempts'],
-                                  self.max_attempts))
-                return True
+            self.insert_errors += len(err.rows)
+            if not err.final:
+                shell.printerr("Failed to import %d rows: %s - %s,  will retry later, attempt %d of %d"
+                               % (len(err.rows), err.name, err.msg, err.attempts, self.max_attempts))
             else:
-                self.add_failed_rows(batch['rows'])
-                shell.printerr("Failed to import %d rows: %s -  given up after %d attempts"
-                               % (len(batch['rows']), err, batch['attempts']))
-                return False
-
-    @staticmethod
-    def is_parse_error(err):
-        """
-        We treat parse errors as unrecoverable and we have different global counters for giving up when
-        a maximum has been reached. We consider value and type errors as parse errors as well since they
-        are typically non recoverable.
-        """
-        return err.startswith('ValueError') or err.startswith('TypeError') or \
-            err.startswith('ParseError') or err.startswith('IndexError')
+                self.add_failed_rows(err.rows)
+                shell.printerr("Failed to import %d rows: %s - %s,  given up after %d attempts"
+                               % (len(err.rows), err.name, err.msg, err.attempts))
 
 
 class ImportTask(CopyTask):
@@ -818,22 +984,14 @@ class ImportTask(CopyTask):
         CopyTask.__init__(self, shell, ks, table, columns, fname, opts, protocol_version, config_file, 'from')
 
         options = self.options
-        self.ingest_rate = options.copy['ingestrate']
-        self.max_attempts = options.copy['maxattempts']
-        self.header = options.copy['header']
         self.skip_columns = [c.strip() for c in self.options.copy['skipcols'].split(',')]
         self.valid_columns = [c for c in self.columns if c not in self.skip_columns]
         self.table_meta = self.shell.get_table_meta(self.ks, self.table)
-        self.batch_id = 0
         self.receive_meter = RateMeter(log_fcn=self.printmsg,
                                        update_interval=options.copy['reportfrequency'],
                                        log_file=options.copy['ratefile'])
-        self.send_meter = RateMeter(log_fcn=None, update_interval=1)
-        self.reader = ImportReader(self)
-        self.import_errors = ImportErrors(self)
-        self.retries = deque([])
-        self.failed = 0
-        self.succeeded = 0
+        self.error_handler = ImportErrorHandler(self)
+        self.feeding_result = None
         self.sent = 0
 
     def make_params(self):
@@ -861,17 +1019,24 @@ class ImportTask(CopyTask):
         self.printmsg("\nStarting copy of %s.%s with columns %s." % (self.ks, self.table, self.valid_columns))
 
         try:
-            self.reader.start()
             params = self.make_params()
 
-            for i in range(self.num_processes):
-                self.processes.append(ImportProcess(params))
+            for i in range(self.num_processes - 1):
+                self.processes.append(ImportProcess(self.update_params(params, i)))
+
+            feeder = FeedingProcess(self.outmsg.channels[-1], self.inmsg.channels[-1],
+                                    self.outmsg.channels[:-1], self.fname, self.options)
+            self.processes.append(feeder)
 
-            for process in self.processes:
-                process.start()
+            self.start_processes()
+
+            pr = profile_on() if PROFILE_ON else None
 
             self.import_records()
 
+            if pr:
+                profile_off(pr, file_name='parent_profile_%d.txt' % (os.getpid(),))
+
         except Exception, exc:
             shell.printerr(str(exc))
             if shell.debug:
@@ -880,9 +1045,22 @@ class ImportTask(CopyTask):
         finally:
             self.close()
 
-    def close(self):
-        CopyTask.close(self)
-        self.reader.close()
+    def send_stdin_rows(self):
+        """
+        We need to pass stdin rows to the feeder process as it is not safe to pickle or share stdin
+        directly (in case of file the child process would close it). This is a very primitive support
+        for STDIN import in that we we won't start reporting progress until STDIN is fully consumed. I
+        think this is reasonable.
+        """
+        shell = self.shell
+
+        self.printmsg("[Use \. on a line by itself to end input]")
+        for row in shell.use_stdin_reader(prompt='[copy] ', until=r'.'):
+            self.outmsg.channels[-1].send(row)
+
+        self.outmsg.channels[-1].send(None)
+        if shell.tty:
+            print
 
     def import_records(self):
         """
@@ -890,114 +1068,137 @@ class ImportTask(CopyTask):
         Send data (batches or retries) up to the max ingest rate. If we are waiting for stuff to
         receive check the incoming queue.
         """
-        reader = self.reader
-
-        while self.has_more_to_send(reader) or self.has_more_to_receive():
-            if self.has_more_to_send(reader):
-                self.send_batches(reader)
+        if not self.fname:
+            self.send_stdin_rows()
 
-            if self.has_more_to_receive():
-                self.receive()
+        while self.feeding_result is None or self.receive_meter.total_records < self.feeding_result.sent:
+            self.receive_results()
 
-            if self.import_errors.max_exceeded() or not self.all_processes_running():
+            if self.error_handler.max_exceeded() or not self.all_processes_running():
                 break
 
-        if self.import_errors.num_rows_failed:
+        if self.error_handler.num_rows_failed:
             self.shell.printerr("Failed to process %d rows; failed rows written to %s" %
-                                (self.import_errors.num_rows_failed,
-                                 self.import_errors.err_file))
+                                (self.error_handler.num_rows_failed,
+                                 self.error_handler.err_file))
 
         if not self.all_processes_running():
             self.shell.printerr("{} child process(es) died unexpectedly, aborting"
                                 .format(self.num_processes - self.num_live_processes()))
+        else:
+            # it is only safe to write to processes if they are all running because the feeder process
+            # at the moment hangs whilst sending messages to a crashed worker process; in future
+            # we could do something about this by using a BoundedSemaphore to keep track of how many messages are
+            # queued on a pipe
+            for i, _ in enumerate(self.processes):
+                self.outmsg.channels[i].send(None)
+
+            if PROFILE_ON:
+                # allow time for worker processes to write profile results (only works if processes received
+                # the poison pill above)
+                time.sleep(5)
 
         self.printmsg("\n%d rows imported from %d files in %s (%d skipped)." %
                       (self.receive_meter.get_total_records(),
-                       self.reader.num_sources,
+                       self.feeding_result.num_sources if self.feeding_result else 0,
                        self.describe_interval(time.time() - self.time_start),
-                       self.reader.skip_rows))
-
-    def has_more_to_receive(self):
-        return (self.succeeded + self.failed) < self.sent
-
-    def has_more_to_send(self, reader):
-        return (not reader.exhausted) or self.retries
+                       self.feeding_result.skip_rows if self.feeding_result else 0))
 
     def all_processes_running(self):
-        return self.num_live_processes() == self.num_processes
+        return self.num_live_processes() == len(self.processes)
 
-    def receive(self):
-        start_time = time.time()
+    def receive_results(self):
+        """
+        Receive results from the worker processes, which will send the number of rows imported
+        or from the feeder process, which will send the number of rows sent when it has finished sending rows.
+        """
+        aggregate_result = ImportProcessResult()
+        try:
+            for result in self.inmsg.recv(timeout=0.1):
+                if isinstance(result, ImportProcessResult):
+                    aggregate_result.imported += result.imported
+                elif isinstance(result, ImportTaskError):
+                    self.error_handler.handle_error(result)
+                elif isinstance(result, FeedingProcessResult):
+                    self.feeding_result = result
+                else:
+                    raise ValueError("Unexpected result: %s" % (result,))
+        finally:
+            self.receive_meter.increment(aggregate_result.imported)
 
-        while time.time() - start_time < 0.001:
-            try:
-                batch, err = self.inmsg.get(timeout=0.00001)
 
-                if err is None:
-                    self.succeeded += batch['imported']
-                    self.receive_meter.increment(batch['imported'])
-                else:
-                    err = str(err)
+class FeedingProcess(mp.Process):
+    """
+    A process that reads from import sources and sends chunks to worker processes.
+    """
+    def __init__(self, inmsg, outmsg, worker_channels, fname, options):
+        mp.Process.__init__(self, target=self.run)
+        self.inmsg = inmsg
+        self.outmsg = outmsg
+        self.worker_channels = worker_channels
+        self.reader = FilesReader(fname, options) if fname else PipeReader(inmsg, options)
+        self.send_meter = RateMeter(log_fcn=None, update_interval=1)
+        self.ingest_rate = options.copy['ingestrate']
+        self.num_worker_processes = options.copy['numprocesses']
+        self.chunk_id = 0
+
+    def run(self):
+        pr = profile_on() if PROFILE_ON else None
 
-                    if self.import_errors.handle_error(err, batch):
-                        self.retries.append(self.reset_batch(batch))
-                    else:
-                        self.failed += len(batch['rows'])
+        self.inner_run()
 
-            except Queue.Empty:
-                pass
+        if pr:
+            profile_off(pr, file_name='feeder_profile_%d.txt' % (os.getpid(),))
 
-    def send_batches(self, reader):
+    def inner_run(self):
         """
         Send one batch per worker process to the queue unless we have exceeded the ingest rate.
         In the export case we queue everything and let the worker processes throttle using max_requests,
-        here we throttle using the ingest rate in the parent process because of memory usage concerns.
-
-        When we have finished reading the csv file, then send any retries.
+        here we throttle using the ingest rate in the feeding process because of memory usage concerns.
+        When finished we send back to the parent process the total number of rows sent.
         """
-        for _ in xrange(self.num_processes):
-            max_rows = self.ingest_rate - self.send_meter.current_record
-            if max_rows <= 0:
-                self.send_meter.maybe_update()
-                break
+        reader = self.reader
+        reader.start()
+        channels = self.worker_channels
+        sent = 0
+
+        while not reader.exhausted:
+            for ch in channels:
+                try:
+                    max_rows = self.ingest_rate - self.send_meter.current_record
+                    if max_rows <= 0:
+                        self.send_meter.maybe_update(sleep=False)
+                        continue
+
+                    rows = reader.read_rows(max_rows)
+                    if rows:
+                        sent += self.send_chunk(ch, rows)
+                except Exception, exc:
+                    self.outmsg.send(ImportTaskError(exc.__class__.__name__, exc.message))
+
+                if reader.exhausted:
+                    break
 
-            if not reader.exhausted:
-                rows = reader.read_rows(max_rows)
-                if rows:
-                    self.sent += self.send_batch(self.new_batch(rows))
-            elif self.retries:
-                batch = self.retries.popleft()
-                if len(batch['rows']) <= max_rows:
-                    self.send_batch(batch)
-                else:
-                    self.send_batch(self.split_batch(batch, batch['rows'][:max_rows]))
-                    self.retries.append(self.split_batch(batch, batch['rows'][max_rows:]))
-            else:
-                break
+        # send back to the parent process the number of rows sent to the worker processes
+        self.outmsg.send(FeedingProcessResult(sent, reader))
+
+        # wait for poison pill (None)
+        self.inmsg.recv()
 
-    def send_batch(self, batch):
-        batch['attempts'] += 1
-        num_rows = len(batch['rows'])
+    def send_chunk(self, ch, rows):
+        self.chunk_id += 1
+        num_rows = len(rows)
         self.send_meter.increment(num_rows)
-        self.outmsg.put(batch)
+        ch.send({'id': self.chunk_id, 'rows': rows, 'imported': 0, 'num_rows_sent': num_rows})
         return num_rows
 
-    def new_batch(self, rows):
-        self.batch_id += 1
-        return self.make_batch(self.batch_id, rows, 0)
-
-    @staticmethod
-    def reset_batch(batch):
-        batch['imported'] = 0
-        return batch
-
-    @staticmethod
-    def split_batch(batch, rows):
-        return ImportTask.make_batch(batch['id'], rows, batch['attempts'])
+    def close(self):
+        self.reader.close()
+        self.inmsg.close()
+        self.outmsg.close()
 
-    @staticmethod
-    def make_batch(batch_id, rows, attempts):
-        return {'id': batch_id, 'rows': rows, 'attempts': attempts, 'imported': 0}
+        for ch in self.worker_channels:
+            ch.close()
 
 
 class ChildProcess(mp.Process):
@@ -1029,6 +1230,7 @@ class ChildProcess(mp.Process):
         self.decimal_sep = options.copy['decimalsep']
         self.thousands_sep = options.copy['thousandssep']
         self.boolean_styles = options.copy['boolstyle']
+        self.max_attempts = options.copy['maxattempts']
         # Here we inject some failures for testing purposes, only if this environment variable is set
         if os.environ.get('CQLSH_COPY_TEST_FAILURES', ''):
             self.test_failures = json.loads(os.environ.get('CQLSH_COPY_TEST_FAILURES', ''))
@@ -1144,7 +1346,6 @@ class ExportProcess(ChildProcess):
         self.encoding = options.copy['encoding']
         self.float_precision = options.copy['float_precision']
         self.nullval = options.copy['nullval']
-        self.max_attempts = options.copy['maxattempts']
         self.max_requests = options.copy['maxrequests']
 
         self.hosts_to_sessions = dict()
@@ -1172,7 +1373,7 @@ class ExportProcess(ChildProcess):
                 time.sleep(0.001)  # 1 millisecond
                 continue
 
-            token_range, info = self.inmsg.get()
+            token_range, info = self.inmsg.recv()
             self.start_request(token_range, info)
 
     @staticmethod
@@ -1190,7 +1391,7 @@ class ExportProcess(ChildProcess):
     def report_error(self, err, token_range=None):
         msg = self.get_error_message(err, print_traceback=self.debug)
         self.printdebugmsg(msg)
-        self.outmsg.put((token_range, Exception(msg)))
+        self.outmsg.send((token_range, Exception(msg)))
 
     def start_request(self, token_range, info):
         """
@@ -1253,7 +1454,8 @@ class ExportProcess(ChildProcess):
             default_retry_policy=ExpBackoffRetryPolicy(self),
             compression=None,
             control_connection_timeout=self.connect_timeout,
-            connect_timeout=self.connect_timeout)
+            connect_timeout=self.connect_timeout,
+            idle_heartbeat_interval=0)
         session = ExportSession(new_cluster, self)
         self.hosts_to_sessions[host] = session
         return session
@@ -1265,7 +1467,7 @@ class ExportProcess(ChildProcess):
                 self.write_rows_to_csv(token_range, rows)
             else:
                 self.write_rows_to_csv(token_range, rows)
-                self.outmsg.put((None, None))
+                self.outmsg.send((None, None))
                 session.complete_request()
 
         def err_callback(err):
@@ -1286,7 +1488,7 @@ class ExportProcess(ChildProcess):
                 writer.writerow(map(self.format_value, row))
 
             data = (output.getvalue(), len(rows))
-            self.outmsg.put((token_range, data))
+            self.outmsg.send((token_range, data))
             output.close()
 
         except Exception, e:
@@ -1376,7 +1578,7 @@ class ImportConversion(object):
     A class for converting strings to values when importing from csv, used by ImportProcess,
     the parent.
     """
-    def __init__(self, parent, table_meta, statement):
+    def __init__(self, parent, table_meta, statement=None):
         self.ks = parent.ks
         self.table = parent.table
         self.columns = parent.valid_columns
@@ -1391,9 +1593,37 @@ class ImportConversion(object):
         self.primary_key_indexes = [self.columns.index(col.name) for col in self.table_meta.primary_key]
         self.partition_key_indexes = [self.columns.index(col.name) for col in self.table_meta.partition_key]
 
+        if statement is None:
+            self.use_prepared_statements = False
+            statement = self._get_primary_key_statement(parent, table_meta)
+        else:
+            self.use_prepared_statements = True
+
         self.proto_version = statement.protocol_version
-        self.cqltypes = dict([(c.name, c.type) for c in statement.column_metadata])
-        self.converters = dict([(c.name, self._get_converter(c.type)) for c in statement.column_metadata])
+
+        # the cql types and converters for the prepared statement, either the full statement or only the primary keys
+        self.cqltypes = [c.type for c in statement.column_metadata]
+        self.converters = [self._get_converter(c.type) for c in statement.column_metadata]
+
+        # the cql types for the entire statement, these are the same as the types above but
+        # only when using prepared statements
+        self.coltypes = [table_meta.columns[name].typestring for name in parent.valid_columns]
+        # these functions are used for non-prepared statements to protect values with quotes if required
+        self.protectors = [protect_value if t in ('ascii', 'text', 'timestamp', 'date', 'time', 'inet') else lambda v: v
+                           for t in self.coltypes]
+
+    @staticmethod
+    def _get_primary_key_statement(parent, table_meta):
+        """
+        We prepare a query statement to find out the types of the partition key columns so we can
+        route the update query to the correct replicas. As far as I understood this is the easiest
+        way to find out the types of the partition columns, we will never use this prepared statement
+        """
+        where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
+        select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(parent.ks),
+                                                         protect_name(parent.table),
+                                                         where_clause)
+        return parent.session.prepare(select_query)
 
     def _get_converter(self, cql_type):
         """
@@ -1581,27 +1811,25 @@ class ImportConversion(object):
 
         return converters.get(cql_type.typename, convert_unknown)
 
-    def get_row_values(self, row):
+    def convert_row(self, row):
         """
-        Parse the row into a list of row values to be returned
+        Convert the row into a list of parsed values if using prepared statements, else simply apply the
+        protection functions to escape values with quotes when required. Also check on the row length and
+        make sure primary partition key values aren't missing.
         """
-        def convert(n, val):
-            try:
-                return self.converters[self.columns[n]](val)
-            except Exception, e:
-                raise ParseError(e.message)
+        converters = self.converters if self.use_prepared_statements else self.protectors
 
-        ret = [None] * len(row)
-        for i, val in enumerate(row):
-            if val != self.nullval:
-                ret[i] = convert(i, val)
-            else:
-                if i in self.primary_key_indexes:
-                    raise ParseError(self.get_null_primary_key_message(i))
+        if len(row) != len(converters):
+            raise ParseError('Invalid row length %d should be %d' % (len(row), len(converters)))
 
-                ret[i] = None
+        for i in self.primary_key_indexes:
+            if row[i] == self.nullval:
+                raise ParseError(self.get_null_primary_key_message(i))
 
-        return ret
+        try:
+            return [conv(val) for conv, val in zip(converters, row)]
+        except Exception, e:
+            raise ParseError(e.message)
 
     def get_null_primary_key_message(self, idx):
         message = "Cannot insert null value for primary key column '%s'." % (self.columns[idx],)
@@ -1610,31 +1838,111 @@ class ImportConversion(object):
                        " the WITH NULL=<marker> option for COPY."
         return message
 
-    def get_row_partition_key_values(self, row):
+    def get_row_partition_key_values_fcn(self):
         """
-        Return a string composed of the partition key values, serialized and binary packed -
-        as expected by metadata.get_replicas(), see also BoundStatement.routing_key.
+        Return a function to convert a row into a string composed of the partition key values serialized
+        and binary packed (the tokens on the ring). Depending on whether we are using prepared statements, we
+        may have to convert the primary key values first, so we have two different serialize_value implementations.
+        We also return different functions depending on how many partition key indexes we have (single or multiple).
+        See also BoundStatement.routing_key.
         """
-        def serialize(n):
-            try:
-                c, v = self.columns[n], row[n]
-                if v == self.nullval:
-                    raise ParseError(self.get_null_primary_key_message(n))
-                return self.cqltypes[c].serialize(self.converters[c](v), self.proto_version)
-            except Exception, e:
-                raise ParseError(e.message)
+        def serialize_value_prepared(n, v):
+            return self.cqltypes[n].serialize(v, self.proto_version)
+
+        def serialize_value_not_prepared(n, v):
+            return self.cqltypes[n].serialize(self.converters[n](v), self.proto_version)
 
         partition_key_indexes = self.partition_key_indexes
-        if len(partition_key_indexes) == 1:
-            return serialize(partition_key_indexes[0])
-        else:
+        serialize = serialize_value_prepared if self.use_prepared_statements else serialize_value_not_prepared
+
+        def serialize_row_single(row):
+            return serialize(partition_key_indexes[0], row[partition_key_indexes[0]])
+
+        def serialize_row_multiple(row):
             pk_values = []
             for i in partition_key_indexes:
-                val = serialize(i)
+                val = serialize(i, row[i])
                 l = len(val)
                 pk_values.append(struct.pack(">H%dsB" % l, l, val, 0))
             return b"".join(pk_values)
 
+        if len(partition_key_indexes) == 1:
+            return serialize_row_single
+        return serialize_row_multiple
+
+
+class TokenMap(object):
+    """
+    A wrapper around the metadata token map to speed things up by caching ring token *values* and
+    replicas. It is very important that we use the token values, which are primitive types, rather
+    than the tokens classes when calling bisect_right() in split_batches(). If we use primitive values,
+    the bisect is done in compiled code whilst with token classes each comparison requires a call
+    into the interpreter to perform the cmp operation defined in Python. A simple test with 1 million bisect
+    operations on an array of 2048 tokens was done in 0.37 seconds with primitives and 2.25 seconds with
+    token classes. This is significant for large datasets because we need to do a bisect for each single row,
+    and if VNODES are used, the size of the token map can get quite large too.
+    """
+    def __init__(self, ks, hostname, local_dc, session):
+
+        self.ks = ks
+        self.hostname = hostname
+        self.local_dc = local_dc
+        self.metadata = session.cluster.metadata
+
+        self._initialize_ring()
+
+        # Note that refresh metadata is disabled by default and we currenlty do not intercept it
+        # If hosts are added, removed or moved during a COPY operation our token map is no longer optimal
+        # However we can cope with hosts going down and up since we filter for replicas that are up when
+        # making each batch
+
+    def _initialize_ring(self):
+        token_map = self.metadata.token_map
+        if token_map is None:
+            self.ring = [0]
+            self.replicas = [(self.metadata.get_host(self.hostname),)]
+            self.pk_to_token_value = lambda pk: 0
+            return
+
+        token_map.rebuild_keyspace(self.ks, build_if_absent=True)
+        tokens_to_hosts = token_map.tokens_to_hosts_by_ks.get(self.ks, None)
+        from_key = token_map.token_class.from_key
+
+        self.ring = [token.value for token in token_map.ring]
+        self.replicas = [tuple(tokens_to_hosts[token]) for token in token_map.ring]
+        self.pk_to_token_value = lambda pk: from_key(pk).value
+
+    @staticmethod
+    def get_ring_pos(ring, val):
+        idx = bisect_right(ring, val)
+        return idx if idx < len(ring) else 0
+
+    def filter_replicas(self, hosts):
+        shuffled = tuple(sorted(hosts, key=lambda k: random.random()))
+        return filter(lambda r: r.is_up and r.datacenter == self.local_dc, shuffled) if hosts else ()
+
+
+class FastTokenAwarePolicy(DCAwareRoundRobinPolicy):
+    """
+    Send to any replicas attached to the query, or else fall back to DCAwareRoundRobinPolicy
+    """
+
+    def __init__(self, local_dc='', used_hosts_per_remote_dc=0):
+        DCAwareRoundRobinPolicy.__init__(self, local_dc, used_hosts_per_remote_dc)
+
+    def make_query_plan(self, working_keyspace=None, query=None):
+        """
+        Extend TokenAwarePolicy.make_query_plan() so that we choose the same replicas in preference
+        and most importantly we avoid repeating the (slow) bisect
+        """
+        replicas = query.replicas if hasattr(query, 'replicas') else []
+        for r in replicas:
+            yield r
+
+        for r in DCAwareRoundRobinPolicy.make_query_plan(self, working_keyspace, query):
+            if r not in replicas:
+                yield r
+
 
 class ImportProcess(ChildProcess):
 
@@ -1650,7 +1958,12 @@ class ImportProcess(ChildProcess):
         self.max_attempts = options.copy['maxattempts']
         self.min_batch_size = options.copy['minbatchsize']
         self.max_batch_size = options.copy['maxbatchsize']
+        self.use_prepared_statements = options.copy['preparedstatements']
+        self.dialect_options = options.dialect
         self._session = None
+        self.query = None
+        self.conv = None
+        self.make_statement = None
 
     @property
     def session(self):
@@ -1661,12 +1974,13 @@ class ImportProcess(ChildProcess):
                 cql_version=self.cql_version,
                 protocol_version=self.protocol_version,
                 auth_provider=self.auth_provider,
-                load_balancing_policy=TokenAwarePolicy(DCAwareRoundRobinPolicy(local_dc=self.local_dc)),
+                load_balancing_policy=FastTokenAwarePolicy(local_dc=self.local_dc),
                 ssl_options=ssl_settings(self.hostname, self.config_file) if self.ssl else None,
                 default_retry_policy=ExpBackoffRetryPolicy(self),
                 compression=None,
                 control_connection_timeout=self.connect_timeout,
-                connect_timeout=self.connect_timeout)
+                connect_timeout=self.connect_timeout,
+                idle_heartbeat_interval=0)
 
             self._session = cluster.connect(self.ks)
             self._session.default_timeout = None
@@ -1674,13 +1988,12 @@ class ImportProcess(ChildProcess):
 
     def run(self):
         try:
-            table_meta = self.session.cluster.metadata.keyspaces[self.ks].tables[self.table]
-            is_counter = ("counter" in [table_meta.columns[name].typestring for name in self.valid_columns])
+            pr = profile_on() if PROFILE_ON else None
 
-            if is_counter:
-                self.run_counter(table_meta)
-            else:
-                self.run_normal(table_meta)
+            self.inner_run(*self.make_params())
+
+            if pr:
+                profile_off(pr, file_name='worker_profile_%d.txt' % (os.getpid(),))
 
         except Exception, exc:
             if self.debug:
@@ -1694,67 +2007,88 @@ class ImportProcess(ChildProcess):
             self._session.cluster.shutdown()
         ChildProcess.close(self)
 
-    def run_counter(self, table_meta):
-        """
-        Main run method for tables that contain counter columns.
-        """
-        query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
-
-        # We prepare a query statement to find out the types of the partition key columns so we can
-        # route the update query to the correct replicas. As far as I understood this is the easiest
-        # way to find out the types of the partition columns, we will never use this prepared statement
-        where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
-        select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(self.ks), protect_name(self.table), where_clause)
-        conv = ImportConversion(self, table_meta, self.session.prepare(select_query))
-
-        while True:
-            batch = self.inmsg.get()
-            try:
-                for b in self.split_batches(batch, conv):
-                    self.send_counter_batch(query, conv, b)
+    def make_params(self):
+        metadata = self.session.cluster.metadata
+        table_meta = metadata.keyspaces[self.ks].tables[self.table]
+
+        prepared_statement = None
+        is_counter = ("counter" in [table_meta.columns[name].typestring for name in self.valid_columns])
+        if is_counter:
+            query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
+            make_statement = self.wrap_make_statement(self.make_counter_batch_statement)
+        elif self.use_prepared_statements:
+            query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
+                                                            protect_name(self.table),
+                                                            ', '.join(protect_names(self.valid_columns),),
+                                                            ', '.join(['?' for _ in self.valid_columns]))
+
+            query = self.session.prepare(query)
+            query.consistency_level = self.consistency_level
+            prepared_statement = query
+            make_statement = self.wrap_make_statement(self.make_prepared_batch_statement)
+        else:
+            query = 'INSERT INTO %s.%s (%s) VALUES (%%s)' % (protect_name(self.ks),
+                                                             protect_name(self.table),
+                                                             ', '.join(protect_names(self.valid_columns),))
+            make_statement = self.wrap_make_statement(self.make_non_prepared_batch_statement)
 
-            except Exception, exc:
-                self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                if self.debug:
-                    traceback.print_exc(exc)
+        conv = ImportConversion(self, table_meta, prepared_statement)
+        tm = TokenMap(self.ks, self.hostname, self.local_dc, self.session)
+        return query, conv, tm, make_statement
 
-    def run_normal(self, table_meta):
+    def inner_run(self, query, conv, tm, make_statement):
         """
-        Main run method for normal tables, i.e. tables that do not contain counter columns.
+        Main run method. Note that we bind self methods that are called inside loops
+        for performance reasons.
         """
-        query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
-                                                        protect_name(self.table),
-                                                        ', '.join(protect_names(self.valid_columns),),
-                                                        ', '.join(['?' for _ in self.valid_columns]))
+        self.query = query
+        self.conv = conv
+        self.make_statement = make_statement
 
-        query_statement = self.session.prepare(query)
-        query_statement.consistency_level = self.consistency_level
-        conv = ImportConversion(self, table_meta, query_statement)
+        convert_rows = self.convert_rows
+        split_into_batches = self.split_into_batches
+        result_callback = self.result_callback
+        err_callback = self.err_callback
+        session = self.session
 
         while True:
-            batch = self.inmsg.get()
+            chunk = self.inmsg.recv()
+            if chunk is None:
+                break
+
             try:
-                for b in self.split_batches(batch, conv):
-                    self.send_normal_batch(conv, query_statement, b)
+                chunk['rows'] = convert_rows(conv, chunk)
+                for replicas, batch in split_into_batches(chunk, conv, tm):
+                    statement = make_statement(query, conv, chunk, batch, replicas)
+                    future = session.execute_async(statement)
+                    future.add_callbacks(callback=result_callback, callback_args=(batch, chunk),
+                                         errback=err_callback, errback_args=(batch, chunk, replicas))
 
             except Exception, exc:
-                self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                if self.debug:
-                    traceback.print_exc(exc)
+                self.report_error(exc, chunk, chunk['rows'])
 
-    def send_counter_batch(self, query_text, conv, batch):
-        if self.test_failures and self.maybe_inject_failures(batch):
-            return
+    def wrap_make_statement(self, inner_make_statement):
+        def make_statement(query, conv, chunk, batch, replicas):
+            try:
+                return inner_make_statement(query, conv, batch, replicas)
+            except Exception, exc:
+                print "Failed to make batch statement: {}".format(exc)
+                self.report_error(exc, chunk, batch['rows'])
+                return None
 
-        error_rows = []
-        batch_statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+        def make_statement_with_failures(query, conv, chunk, batch, replicas):
+            failed_batch = self.maybe_inject_failures(batch)
+            if failed_batch:
+                return failed_batch
+            return make_statement(query, conv, chunk, batch, replicas)
 
-        for r in batch['rows']:
-            row = self.filter_row_values(r)
-            if len(row) != len(self.valid_columns):
-                error_rows.append(row)
-                continue
+        return make_statement_with_failures if self.test_failures else make_statement
 
+    def make_counter_batch_statement(self, query, conv, batch, replicas):
+        statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+        statement.replicas = replicas
+        statement.keyspace = self.ks
+        for row in batch['rows']:
             where_clause = []
             set_clause = []
             for i, value in enumerate(row):
@@ -1763,65 +2097,61 @@ class ImportProcess(ChildProcess):
                 else:
                     set_clause.append("%s=%s+%s" % (self.valid_columns[i], self.valid_columns[i], value))
 
-            full_query_text = query_text % (','.join(set_clause), ' AND '.join(where_clause))
-            batch_statement.add(full_query_text)
-
-        self.execute_statement(batch_statement, batch)
+            full_query_text = query % (','.join(set_clause), ' AND '.join(where_clause))
+            statement.add(full_query_text)
+        return statement
 
-        if error_rows:
-            self.outmsg.put((ImportTask.split_batch(batch, error_rows),
-                            '%s - %s' % (ParseError.__name__, "Failed to parse one or more rows")))
+    def make_prepared_batch_statement(self, query, _, batch, replicas):
+        """
+        Return a batch statement. This is an optimized version of:
 
-    def send_normal_batch(self, conv, query_statement, batch):
-        if self.test_failures and self.maybe_inject_failures(batch):
-            return
+            statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+            for row in batch['rows']:
+                statement.add(query, row)
 
-        good_rows, converted_rows, errors = self.convert_rows(conv, batch['rows'])
+        We could optimize further by removing bound_statements altogether but we'd have to duplicate much
+        more driver's code (BoundStatement.bind()).
+        """
+        statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+        statement.replicas = replicas
+        statement.keyspace = self.ks
+        statement._statements_and_parameters = [(True, query.query_id, query.bind(r).values) for r in batch['rows']]
+        return statement
 
-        if converted_rows:
-            try:
-                statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
-                for row in converted_rows:
-                    statement.add(query_statement, row)
-                self.execute_statement(statement, ImportTask.split_batch(batch, good_rows))
-            except Exception, exc:
-                self.err_callback(exc, ImportTask.split_batch(batch, good_rows))
+    def make_non_prepared_batch_statement(self, query, _, batch, replicas):
+        statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+        statement.replicas = replicas
+        statement.keyspace = self.ks
+        statement._statements_and_parameters = [(False, query % (','.join(r),), ()) for r in batch['rows']]
+        return statement
 
-        if errors:
-            for msg, rows in errors.iteritems():
-                self.outmsg.put((ImportTask.split_batch(batch, rows),
-                                '%s - %s' % (ParseError.__name__, msg)))
-
-    def convert_rows(self, conv, rows):
+    def convert_rows(self, conv, chunk):
         """
-        Try to convert each row. If conversion is OK then add the converted result to converted_rows
-        and the original string to good_rows. Else add the original string to error_rows. Return the three
-        arrays.
+        Return converted rows and report any errors during conversion.
         """
-        good_rows = []
-        errors = defaultdict(list)
-        converted_rows = []
+        def filter_row_values(row):
+            return [v for i, v in enumerate(row) if i not in self.skip_column_indexes]
 
-        for r in rows:
-            row = self.filter_row_values(r)
-            if len(row) != len(self.valid_columns):
-                msg = 'Invalid row length %d should be %d' % (len(row), len(self.valid_columns))
-                errors[msg].append(row)
-                continue
+        if self.skip_column_indexes:
+            rows = [filter_row_values(r) for r in list(csv.reader(chunk['rows'], **self.dialect_options))]
+        else:
+            rows = list(csv.reader(chunk['rows'], **self.dialect_options))
 
-            try:
-                converted_rows.append(conv.get_row_values(row))
-                good_rows.append(row)
-            except ParseError, err:
-                errors[err.message].append(row)
+        errors = defaultdict(list)
 
-        return good_rows, converted_rows, errors
+        def convert_row(r):
+            try:
+                return conv.convert_row(r)
+            except Exception, err:
+                errors[err.message].append(r)
+                return None
 
-    def filter_row_values(self, row):
-        if not self.skip_column_indexes:
-            return row
+        converted_rows = filter(None, [convert_row(r) for r in rows])
 
-        return [v for i, v in enumerate(row) if i not in self.skip_column_indexes]
+        if errors:
+            for msg, rows in errors.iteritems():
+                self.report_error(ParseError(msg), chunk, rows)
+        return converted_rows
 
     def maybe_inject_failures(self, batch):
         """
@@ -1836,86 +2166,94 @@ class ImportProcess(ChildProcess):
                 if batch['attempts'] < failing_batch['failures']:
                     statement = SimpleStatement("INSERT INTO badtable (a, b) VALUES (1, 2)",
                                                 consistency_level=self.consistency_level)
-                    self.execute_statement(statement, batch)
-                    return True
+                    return statement
 
         if 'exit_batch' in self.test_failures:
             exit_batch = self.test_failures['exit_batch']
             if exit_batch['id'] == batch['id']:
                 sys.exit(1)
 
-        return False  # carry on as normal
+        return None  # carry on as normal
 
-    def execute_statement(self, statement, batch):
-        future = self.session.execute_async(statement)
-        future.add_callbacks(callback=self.result_callback, callback_args=(batch, ),
-                             errback=self.err_callback, errback_args=(batch, ))
+    @staticmethod
+    def make_batch(batch_id, rows, attempts=1):
+        return {'id': batch_id, 'rows': rows, 'attempts': attempts}
 
-    def split_batches(self, batch, conv):
+    def split_into_batches(self, chunk, conv, tm):
         """
-        Batch rows by partition key, if there are at least min_batch_size (2)
-        rows with the same partition key. These batches can be as big as they want
-        since this translates to a single insert operation server side.
-
-        If there are less than min_batch_size rows for a partition, work out the
-        first replica for this partition and add the rows to replica left-over rows.
-
-        Then batch the left-overs of each replica up to max_batch_size.
+        Batch rows by ring position or replica.
+        If there are at least min_batch_size rows for a ring position then split these rows into
+        groups of max_batch_size and send a batch for each group, using all replicas for this ring position.
+        Otherwise, we are forced to batch by replica, and here unfortunately we can only choose one replica to
+        guarantee common replicas across partition keys. We are typically able
+        to batch by ring position for small clusters or when VNODES are not used. For large clusters with VNODES
+        it may not be possible, in this case it helps to increase the CHUNK SIZE but up to a limit, otherwise
+        we may choke the cluster.
         """
-        rows_by_pk = defaultdict(list)
+
+        rows_by_ring_pos = defaultdict(list)
         errors = defaultdict(list)
 
-        for row in batch['rows']:
+        min_batch_size = self.min_batch_size
+        max_batch_size = self.max_batch_size
+        ring = tm.ring
+
+        get_row_partition_key_values = conv.get_row_partition_key_values_fcn()
+        pk_to_token_value = tm.pk_to_token_value
+        get_ring_pos = tm.get_ring_pos
+        make_batch = self.make_batch
+
+        for row in chunk['rows']:
             try:
-                pk = conv.get_row_partition_key_values(row)
-                rows_by_pk[pk].append(row)
-            except ParseError, e:
+                pk = get_row_partition_key_values(row)
+                rows_by_ring_pos[get_ring_pos(ring, pk_to_token_value(pk))].append(row)
+            except Exception, e:
                 errors[e.message].append(row)
 
         if errors:
             for msg, rows in errors.iteritems():
-                self.outmsg.put((ImportTask.split_batch(batch, rows),
-                                 '%s - %s' % (ParseError.__name__, msg)))
+                self.report_error(ParseError(msg), chunk, rows)
 
+        replicas = tm.replicas
+        filter_replicas = tm.filter_replicas
         rows_by_replica = defaultdict(list)
-        for pk, rows in rows_by_pk.iteritems():
-            if len(rows) >= self.min_batch_size:
-                yield ImportTask.make_batch(batch['id'], rows, batch['attempts'])
+        for ring_pos, rows in rows_by_ring_pos.iteritems():
+            if len(rows) > min_batch_size:
+                for i in xrange(0, len(rows), max_batch_size):
+                    yield filter_replicas(replicas[ring_pos]), make_batch(chunk['id'], rows[i:i + max_batch_size])
             else:
-                replica = self.get_replica(pk)
-                rows_by_replica[replica].extend(rows)
-
-        for replica, rows in rows_by_replica.iteritems():
-            for b in self.batches(rows, batch):
-                yield b
-
-    def get_replica(self, pk):
-        """
-        Return the first replica or the host we are already connected to if there are no local
-        replicas that are up. We always use the first replica to match the replica chosen by the driver
-        TAR, see TokenAwarePolicy.make_query_plan().
-        """
-        metadata = self.session.cluster.metadata
-        replicas = filter(lambda r: r.is_up and r.datacenter == self.local_dc, metadata.get_replicas(self.ks, pk))
-        ret = replicas[0].address if len(replicas) > 0 else self.hostname
-        return ret
-
-    def batches(self, rows, batch):
-        """
-        Split rows into batches of max_batch_size
-        """
-        for i in xrange(0, len(rows), self.max_batch_size):
-            yield ImportTask.make_batch(batch['id'], rows[i:i + self.max_batch_size], batch['attempts'])
-
-    def result_callback(self, _, batch):
-        batch['imported'] = len(batch['rows'])
-        batch['rows'] = []  # no need to resend these, just send the count in 'imported'
-        self.outmsg.put((batch, None))
-
-    def err_callback(self, response, batch):
-        self.outmsg.put((batch, '%s - %s' % (response.__class__.__name__, response.message)))
+                # select only the first valid replica to guarantee more overlap or none at all
+                rows_by_replica[filter_replicas(replicas[ring_pos])[:1]].extend(rows)
+
+        # Now send the batches by replica
+        for replicas, rows in rows_by_replica.iteritems():
+            for i in xrange(0, len(rows), max_batch_size):
+                yield replicas, make_batch(chunk['id'], rows[i:i + max_batch_size])
+
+    def result_callback(self, _, batch, chunk):
+        self.update_chunk(batch['rows'], chunk)
+
+    def err_callback(self, response, batch, chunk, replicas):
+        err_is_final = batch['attempts'] >= self.max_attempts
+        self.report_error(response, chunk, batch['rows'], batch['attempts'], err_is_final)
+        if not err_is_final:
+            batch['attempts'] += 1
+            statement = self.make_statement(self.query, self.conv, chunk, batch, replicas)
+            future = self.session.execute_async(statement)
+            future.add_callbacks(callback=self.result_callback, callback_args=(batch, chunk),
+                                 errback=self.err_callback, errback_args=(batch, chunk, replicas))
+
+    def report_error(self, err, chunk, rows=None, attempts=1, final=True):
         if self.debug:
-            traceback.print_exc(response)
+            traceback.print_exc(err)
+        self.outmsg.send(ImportTaskError(err.__class__.__name__, err.message, rows, attempts, final))
+        if final:
+            self.update_chunk(rows, chunk)
+
+    def update_chunk(self, rows, chunk):
+        chunk['imported'] += len(rows)
+        if chunk['imported'] == chunk['num_rows_sent']:
+            self.outmsg.send(ImportProcessResult(chunk['num_rows_sent']))
 
 
 class RateMeter(object):
@@ -1937,11 +2275,19 @@ class RateMeter(object):
         self.current_record += n
         self.maybe_update()
 
-    def maybe_update(self):
+    def maybe_update(self, sleep=False):
+        if self.current_record == 0:
+            return
+
         new_checkpoint_time = time.time()
-        if new_checkpoint_time - self.last_checkpoint_time >= self.update_interval:
+        time_difference = new_checkpoint_time - self.last_checkpoint_time
+        if time_difference >= self.update_interval:
             self.update(new_checkpoint_time)
             self.log_message()
+        elif sleep:
+            remaining_time = time_difference - self.update_interval
+            if remaining_time > 0.000001:
+                time.sleep(remaining_time)
 
     def update(self, new_checkpoint_time):
         time_difference = new_checkpoint_time - self.last_checkpoint_time

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/pylib/cqlshlib/util.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/util.py b/pylib/cqlshlib/util.py
index 281aad6..3ee128d 100644
--- a/pylib/cqlshlib/util.py
+++ b/pylib/cqlshlib/util.py
@@ -23,6 +23,12 @@ from itertools import izip
 from datetime import timedelta, tzinfo
 from StringIO import StringIO
 
+try:
+    from line_profiler import LineProfiler
+    HAS_LINE_PROFILER = True
+except ImportError:
+    HAS_LINE_PROFILER = False
+
 ZERO = timedelta(0)
 
 
@@ -126,18 +132,35 @@ def get_file_encoding_bomsize(filename):
     else:
         file_encoding, size = "utf-8", 0
 
-    return (file_encoding, size)
+    return file_encoding, size
+
 
+def profile_on(fcn_names=None):
+    if fcn_names and HAS_LINE_PROFILER:
+        pr = LineProfiler()
+        for fcn_name in fcn_names:
+            pr.add_function(fcn_name)
+        pr.enable()
+        return pr
 
-def profile_on():
     pr = cProfile.Profile()
     pr.enable()
     return pr
 
 
-def profile_off(pr):
+def profile_off(pr, file_name):
     pr.disable()
     s = StringIO()
-    ps = pstats.Stats(pr, stream=s).sort_stats('cumulative')
-    ps.print_stats()
-    print s.getvalue()
+
+    if HAS_LINE_PROFILER and isinstance(pr, LineProfiler):
+        pr.print_stats(s)
+    else:
+        ps = pstats.Stats(pr, stream=s).sort_stats('cumulative')
+        ps.print_stats()
+
+    ret = s.getvalue()
+    if file_name:
+        with open(file_name, 'w') as f:
+            print "Writing to %s\n" % (f.name, )
+            f.write(ret)
+    return ret

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/pylib/setup.py
----------------------------------------------------------------------
diff --git a/pylib/setup.py b/pylib/setup.py
index 704d077..3654502 100755
--- a/pylib/setup.py
+++ b/pylib/setup.py
@@ -16,9 +16,11 @@
 # limitations under the License.
 
 from distutils.core import setup
+from Cython.Build import cythonize
 
 setup(
     name="cassandra-pylib",
     description="Cassandra Python Libraries",
     packages=["cqlshlib"],
+    ext_modules=cythonize("cqlshlib/copyutil.py"),
 )


[04/23] cassandra git commit: COPY FROM on large datasets: fix progress report and debug performance

Posted by sl...@apache.org.
COPY FROM on large datasets: fix progress report and debug performance

patch by Stefania Alborghetti; reviewed by Adam Holmberg for CASSANDRA-11053


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c3d2f26f
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c3d2f26f
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c3d2f26f

Branch: refs/heads/cassandra-3.0
Commit: c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc
Parents: 0129f70
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Thu Jan 28 14:31:55 2016 +0800
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Mar 8 10:19:13 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                |    1 +
 bin/cqlsh                  |   28 +-
 pylib/cqlshlib/copyutil.py | 1160 +++++++++++++++++++++++++--------------
 pylib/cqlshlib/util.py     |   35 +-
 pylib/setup.py             |    2 +
 5 files changed, 809 insertions(+), 417 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index eed9035..d6b085c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.14
+ * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
  * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
  * Don't remove FailureDetector history on removeEndpoint (CASSANDRA-10371)
  * Only notify if repair status changed (CASSANDRA-11172)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/bin/cqlsh
----------------------------------------------------------------------
diff --git a/bin/cqlsh b/bin/cqlsh
index 7a39636..374e588 100755
--- a/bin/cqlsh
+++ b/bin/cqlsh
@@ -417,7 +417,7 @@ COPY_COMMON_OPTIONS = ['DELIMITER', 'QUOTE', 'ESCAPE', 'HEADER', 'NULL', 'DATETI
                        'MAXATTEMPTS', 'REPORTFREQUENCY', 'DECIMALSEP', 'THOUSANDSSEP', 'BOOLSTYLE',
                        'NUMPROCESSES', 'CONFIGFILE', 'RATEFILE']
 COPY_FROM_OPTIONS = ['CHUNKSIZE', 'INGESTRATE', 'MAXBATCHSIZE', 'MINBATCHSIZE', 'MAXROWS',
-                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE']
+                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE', 'PREPAREDSTATEMENTS']
 COPY_TO_OPTIONS = ['ENCODING', 'PAGESIZE', 'PAGETIMEOUT', 'BEGINTOKEN', 'ENDTOKEN', 'MAXOUTPUTSIZE', 'MAXREQUESTS']
 
 
@@ -533,8 +533,23 @@ def insert_driver_hooks():
 
 def extend_cql_deserialization():
     """
-    The python driver returns BLOBs as string, but we expect them as bytearrays
+    The python driver returns BLOBs as string, but we expect them as bytearrays; therefore we change
+    the implementation of cassandra.cqltypes.BytesType.deserialize.
+
+    The deserializers package exists only when the driver has been compiled with cython extensions and
+    cassandra.deserializers.DesBytesType replaces cassandra.cqltypes.BytesType.deserialize.
+
+    DesBytesTypeByteArray is a fast deserializer that converts blobs into bytearrays but it was
+    only introduced recently (3.1.0). If it is available we use it, otherwise we remove
+    cassandra.deserializers.DesBytesType so that we fall back onto cassandra.cqltypes.BytesType.deserialize
+    just like in the case where no cython extensions are present.
     """
+    if hasattr(cassandra, 'deserializers'):
+        if hasattr(cassandra.deserializers, 'DesBytesTypeByteArray'):
+            cassandra.deserializers.DesBytesType = cassandra.deserializers.DesBytesTypeByteArray
+        else:
+            del cassandra.deserializers.DesBytesType
+
     cassandra.cqltypes.BytesType.deserialize = staticmethod(lambda byts, protocol_version: bytearray(byts))
     cassandra.cqltypes.CassandraType.support_empty_values = True
 
@@ -1534,9 +1549,9 @@ class Shell(cmd.Cmd):
 
         Available COPY FROM options and defaults:
 
-          CHUNKSIZE=1000          - the size of chunks passed to worker processes
+          CHUNKSIZE=5000          - the size of chunks passed to worker processes
           INGESTRATE=100000       - an approximate ingest rate in rows per second
-          MINBATCHSIZE=2          - the minimum size of an import batch
+          MINBATCHSIZE=10         - the minimum size of an import batch
           MAXBATCHSIZE=20         - the maximum size of an import batch
           MAXROWS=-1              - the maximum number of rows, -1 means no maximum
           SKIPROWS=0              - the number of rows to skip
@@ -1545,6 +1560,11 @@ class Shell(cmd.Cmd):
           MAXINSERTERRORS=-1      - the maximum global number of insert errors, -1 means no maximum
           ERRFILE=''              - a file where to store all rows that could not be imported, by default this is
                                     import_ks_table.err where <ks> is your keyspace and <table> is your table name.
+          PREPAREDSTATEMENTS=True - whether to use prepared statements when importing, by default True. Set this to
+                                    False if you don't mind shifting data parsing to the cluster. The cluster will also
+                                    have to compile every batch statement. For large and oversized clusters
+                                    this will result in a faster import but for smaller clusters it may generate
+                                    timeouts.
 
         Available COPY TO options and defaults:
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/pylib/cqlshlib/copyutil.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/copyutil.py b/pylib/cqlshlib/copyutil.py
index f9e4a85..cd03765 100644
--- a/pylib/cqlshlib/copyutil.py
+++ b/pylib/cqlshlib/copyutil.py
@@ -1,3 +1,5 @@
+# cython: profile=True
+
 # Licensed to the Apache Software Foundation (ASF) under one
 # or more contributor license agreements.  See the NOTICE file
 # distributed with this work for additional information
@@ -21,25 +23,29 @@ import json
 import glob
 import multiprocessing as mp
 import os
-import Queue
+import platform
+import random
 import re
 import struct
 import sys
 import time
 import traceback
 
+from bisect import bisect_right
 from calendar import timegm
-from collections import defaultdict, deque, namedtuple
+from collections import defaultdict, namedtuple
 from decimal import Decimal
 from random import randrange
 from StringIO import StringIO
+from select import select
 from threading import Lock
 from uuid import UUID
+from util import profile_on, profile_off
 
 from cassandra.cluster import Cluster
 from cassandra.cqltypes import ReversedType, UserType
-from cassandra.metadata import protect_name, protect_names
-from cassandra.policies import RetryPolicy, WhiteListRoundRobinPolicy, TokenAwarePolicy, DCAwareRoundRobinPolicy
+from cassandra.metadata import protect_name, protect_names, protect_value
+from cassandra.policies import RetryPolicy, WhiteListRoundRobinPolicy, DCAwareRoundRobinPolicy
 from cassandra.query import BatchStatement, BatchType, SimpleStatement, tuple_factory
 from cassandra.util import Date, Time
 
@@ -48,6 +54,10 @@ from displaying import NO_COLOR_MAP
 from formatting import format_value_default, EMPTY, get_formatter
 from sslhandling import ssl_settings
 
+PROFILE_ON = False
+STRACE_ON = False
+IS_LINUX = platform.system() == 'Linux'
+
 CopyOptions = namedtuple('CopyOptions', 'copy dialect unrecognized')
 
 
@@ -59,6 +69,81 @@ def safe_normpath(fname):
     return os.path.normpath(os.path.expanduser(fname)) if fname else fname
 
 
+class OneWayChannel(object):
+    """
+    A one way pipe protected by two process level locks, one for reading and one for writing.
+    """
+    def __init__(self):
+        self.reader, self.writer = mp.Pipe(duplex=False)
+        self.rlock = mp.Lock()
+        self.wlock = mp.Lock()
+
+    def send(self, obj):
+        with self.wlock:
+            self.writer.send(obj)
+
+    def recv(self):
+        with self.rlock:
+            return self.reader.recv()
+
+    def close(self):
+        self.reader.close()
+        self.writer.close()
+
+
+class OneWayChannels(object):
+    """
+    A group of one way channels.
+    """
+    def __init__(self, num_channels):
+        self.channels = [OneWayChannel() for _ in xrange(num_channels)]
+        self._readers = [ch.reader for ch in self.channels]
+        self._rlocks = [ch.rlock for ch in self.channels]
+        self._rlocks_by_readers = dict([(ch.reader, ch.rlock) for ch in self.channels])
+        self.num_channels = num_channels
+
+        self.recv = self.recv_select if IS_LINUX else self.recv_polling
+
+    def recv_select(self, timeout):
+        """
+        Implementation of the recv method for Linux, where select is available. Receive an object from
+        all pipes that are ready for reading without blocking.
+        """
+        readable, _, _ = select(self._readers, [], [], timeout)
+        for r in readable:
+            with self._rlocks_by_readers[r]:
+                try:
+                    yield r.recv()
+                except EOFError:
+                    continue
+
+    def recv_polling(self, timeout):
+        """
+        Implementation of the recv method for platforms where select() is not available for pipes.
+        We poll on all of the readers with a very small timeout. We stop when the timeout specified
+        has been received but we may exceed it since we check all processes during each sweep.
+        """
+        start = time.time()
+        while True:
+            for i, r in enumerate(self._readers):
+                with self._rlocks[i]:
+                    if r.poll(0.000000001):
+                        try:
+                            yield r.recv()
+                        except EOFError:
+                            continue
+
+            if time.time() - start > timeout:
+                break
+
+    def close(self):
+        for ch in self.channels:
+            try:
+                ch.close()
+            except:
+                pass
+
+
 class CopyTask(object):
     """
     A base class for ImportTask and ExportTask
@@ -72,15 +157,18 @@ class CopyTask(object):
         self.protocol_version = protocol_version
         self.config_file = config_file
         # do not display messages when exporting to STDOUT
-        self.printmsg = self._printmsg if self.fname is not None or direction == 'in' else lambda _, eol='\n': None
+        self.printmsg = self._printmsg if self.fname is not None or direction == 'from' else lambda _, eol='\n': None
         self.options = self.parse_options(opts, direction)
 
         self.num_processes = self.options.copy['numprocesses']
+        if direction == 'in':
+            self.num_processes += 1  # add the feeder process
+
         self.printmsg('Using %d child processes' % (self.num_processes,))
 
         self.processes = []
-        self.inmsg = mp.Queue()
-        self.outmsg = mp.Queue()
+        self.inmsg = OneWayChannels(self.num_processes)
+        self.outmsg = OneWayChannels(self.num_processes)
 
         self.columns = CopyTask.get_columns(shell, ks, table, columns)
         self.time_start = time.time()
@@ -166,10 +254,10 @@ class CopyTask(object):
         copy_options['maxattempts'] = int(opts.pop('maxattempts', 5))
         copy_options['dtformats'] = opts.pop('datetimeformat', shell.display_time_format)
         copy_options['float_precision'] = shell.display_float_precision
-        copy_options['chunksize'] = int(opts.pop('chunksize', 1000))
+        copy_options['chunksize'] = int(opts.pop('chunksize', 5000))
         copy_options['ingestrate'] = int(opts.pop('ingestrate', 100000))
         copy_options['maxbatchsize'] = int(opts.pop('maxbatchsize', 20))
-        copy_options['minbatchsize'] = int(opts.pop('minbatchsize', 2))
+        copy_options['minbatchsize'] = int(opts.pop('minbatchsize', 10))
         copy_options['reportfrequency'] = float(opts.pop('reportfrequency', 0.25))
         copy_options['consistencylevel'] = shell.consistency_level
         copy_options['decimalsep'] = opts.pop('decimalsep', '.')
@@ -186,6 +274,7 @@ class CopyTask(object):
         copy_options['errfile'] = safe_normpath(opts.pop('errfile', 'import_%s_%s.err' % (self.ks, self.table,)))
         copy_options['ratefile'] = safe_normpath(opts.pop('ratefile', ''))
         copy_options['maxoutputsize'] = int(opts.pop('maxoutputsize', '-1'))
+        copy_options['preparedstatements'] = bool(opts.pop('preparedstatements', 'true').lower() == 'true')
 
         self.check_options(copy_options)
         return CopyOptions(copy=copy_options, dialect=dialect_options, unrecognized=opts)
@@ -206,14 +295,17 @@ class CopyTask(object):
     def get_num_processes(cap):
         """
         Pick a reasonable number of child processes. We need to leave at
-        least one core for the parent process.  This doesn't necessarily
-        need to be capped, but 4 is currently enough to keep
-        a single local Cassandra node busy so we use this for import, whilst
-        for export we use 16 since we can connect to multiple Cassandra nodes.
-        Eventually this parameter will become an option.
+        least one core for the parent process.
+        """
+        return max(1, min(cap, CopyTask.get_num_cores() - 1))
+
+    @staticmethod
+    def get_num_cores():
+        """
+        Return the number of cores if available.
         """
         try:
-            return max(1, min(cap, mp.cpu_count() - 1))
+            return mp.cpu_count()
         except NotImplementedError:
             return 1
 
@@ -244,28 +336,40 @@ class CopyTask(object):
         return shell.get_column_names(ks, table) if not columns else columns
 
     def close(self):
-        for process in self.processes:
-            process.terminate()
-
+        self.stop_processes()
         self.inmsg.close()
         self.outmsg.close()
 
     def num_live_processes(self):
         return sum(1 for p in self.processes if p.is_alive())
 
+    @staticmethod
+    def get_pid():
+        return os.getpid() if hasattr(os, 'getpid') else None
+
+    @staticmethod
+    def trace_process(pid):
+        if pid and STRACE_ON:
+            os.system("strace -vvvv -c -o strace.{pid}.out -e trace=all -p {pid}&".format(pid=pid))
+
+    def start_processes(self):
+        for i, process in enumerate(self.processes):
+            process.start()
+            self.trace_process(process.pid)
+
+        self.trace_process(self.get_pid())
+
+    def stop_processes(self):
+        for process in self.processes:
+            process.terminate()
+
     def make_params(self):
         """
         Return a dictionary of parameters to be used by the worker processes.
         On Windows this dictionary must be pickle-able.
-
-        inmsg is the message queue flowing from parent to child process, so outmsg from the parent point
-        of view and, vice-versa,  outmsg is the message queue flowing from child to parent, so inmsg
-        from the parent point of view, hence the two are swapped below.
         """
         shell = self.shell
-        return dict(inmsg=self.outmsg,  # see comment above
-                    outmsg=self.inmsg,  # see comment above
-                    ks=self.ks,
+        return dict(ks=self.ks,
                     table=self.table,
                     local_dc=self.local_dc,
                     columns=self.columns,
@@ -281,6 +385,17 @@ class CopyTask(object):
                     debug=shell.debug
                     )
 
+    def update_params(self, params, i):
+        """
+        Add the communication channels to the parameters to be passed to the worker process:
+            inmsg is the message queue flowing from parent to child process, so outmsg from the parent point
+            of view and, vice-versa,  outmsg is the message queue flowing from child to parent, so inmsg
+            from the parent point of view, hence the two are swapped below.
+        """
+        params['inmsg'] = self.outmsg.channels[i]
+        params['outmsg'] = self.inmsg.channels[i]
+        return params
+
 
 class ExportWriter(object):
     """
@@ -414,10 +529,9 @@ class ExportTask(CopyTask):
 
         params = self.make_params()
         for i in xrange(self.num_processes):
-            self.processes.append(ExportProcess(params))
+            self.processes.append(ExportProcess(self.update_params(params, i)))
 
-        for process in self.processes:
-            process.start()
+        self.start_processes()
 
         try:
             self.export_records(ranges)
@@ -468,11 +582,12 @@ class ExportTask(CopyTask):
 
             return ret
 
-        def make_range_data(replicas=[]):
+        def make_range_data(replicas=None):
             hosts = []
-            for r in replicas:
-                if r.is_up and r.datacenter == local_dc:
-                    hosts.append(r.address)
+            if replicas:
+                for r in replicas:
+                    if r.is_up and r.datacenter == local_dc:
+                        hosts.append(r.address)
             if not hosts:
                 hosts.append(hostname)  # fallback to default host if no replicas in current dc
             return {'hosts': tuple(hosts), 'attempts': 0, 'rows': 0}
@@ -542,10 +657,13 @@ class ExportTask(CopyTask):
             return None
 
     def send_work(self, ranges, tokens_to_send):
+        i = 0
         for token_range in tokens_to_send:
-            self.outmsg.put((token_range, ranges[token_range]))
+            self.outmsg.channels[i].send((token_range, ranges[token_range]))
             ranges[token_range]['attempts'] += 1
 
+            i = i + 1 if i < self.num_processes - 1 else 0
+
     def export_records(self, ranges):
         """
         Send records to child processes and monitor them by collecting their results
@@ -568,8 +686,7 @@ class ExportTask(CopyTask):
         succeeded = 0
         failed = 0
         while (failed + succeeded) < total_requests and self.num_live_processes() == num_processes:
-            try:
-                token_range, result = self.inmsg.get(timeout=1.0)
+            for token_range, result in self.inmsg.recv(timeout=0.1):
                 if token_range is None and result is None:  # a request has finished
                     succeeded += 1
                 elif isinstance(result, Exception):  # an error occurred
@@ -594,8 +711,6 @@ class ExportTask(CopyTask):
                     self.writer.write(data, num)
                     meter.increment(n=num)
                     ranges[token_range]['rows'] += num
-            except Queue.Empty:
-                pass
 
         if self.num_live_processes() < len(processes):
             for process in processes:
@@ -612,7 +727,7 @@ class ExportTask(CopyTask):
                        self.describe_interval(time.time() - self.time_start)))
 
 
-class ImportReader(object):
+class FilesReader(object):
     """
     A wrapper around a csv reader to keep track of when we have
     exhausted reading input files. We are passed a comma separated
@@ -620,18 +735,15 @@ class ImportReader(object):
     We generate a source generator and we read each source one
     by one.
     """
-    def __init__(self, task):
-        self.shell = task.shell
-        self.options = task.options
-        self.printmsg = task.printmsg
-        self.chunk_size = self.options.copy['chunksize']
-        self.header = self.options.copy['header']
-        self.max_rows = self.options.copy['maxrows']
-        self.skip_rows = self.options.copy['skiprows']
-        self.sources = self.get_source(task.fname)
+    def __init__(self, fname, options):
+        self.chunk_size = options.copy['chunksize']
+        self.header = options.copy['header']
+        self.max_rows = options.copy['maxrows']
+        self.skip_rows = options.copy['skiprows']
+        self.fname = fname
+        self.sources = None  # must be created later due to pickle problems on Windows
         self.num_sources = 0
         self.current_source = None
-        self.current_reader = None
         self.num_read = 0
 
     def get_source(self, paths):
@@ -640,35 +752,33 @@ class ImportReader(object):
          wrapping the source input, file name and a boolean indicating
          if it requires closing.
         """
-        shell = self.shell
-        LineSource = namedtuple('LineSource', 'input close fname')
-
         def make_source(fname):
             try:
-                ret = LineSource(input=open(fname, 'rb'), close=True, fname=fname)
-                return ret
+                return open(fname, 'rb')
             except IOError, e:
-                shell.printerr("Can't open %r for reading: %s" % (fname, e))
+                self.printmsg("Can't open %r for reading: %s" % (fname, e))
                 return None
 
-        if paths is None:
-            self.printmsg("[Use \. on a line by itself to end input]")
-            yield LineSource(input=shell.use_stdin_reader(prompt='[copy] ', until=r'\.'), close=False, fname='')
-        else:
-            for path in paths.split(','):
-                path = path.strip()
-                if os.path.isfile(path):
-                    yield make_source(path)
-                else:
-                    for f in glob.glob(path):
-                        yield (make_source(f))
+        for path in paths.split(','):
+            path = path.strip()
+            if os.path.isfile(path):
+                yield make_source(path)
+            else:
+                for f in glob.glob(path):
+                    yield (make_source(f))
+
+    @staticmethod
+    def printmsg(msg, eol='\n'):
+        sys.stdout.write(msg + eol)
+        sys.stdout.flush()
 
     def start(self):
+        self.sources = self.get_source(self.fname)
         self.next_source()
 
     @property
     def exhausted(self):
-        return not self.current_reader
+        return not self.current_source
 
     def next_source(self):
         """
@@ -679,40 +789,34 @@ class ImportReader(object):
         while self.current_source is None:
             try:
                 self.current_source = self.sources.next()
-                if self.current_source and self.current_source.fname:
+                if self.current_source:
                     self.num_sources += 1
             except StopIteration:
                 return False
 
         if self.header:
-            self.current_source.input.next()
+            self.current_source.next()
 
-        self.current_reader = csv.reader(self.current_source.input, **self.options.dialect)
         return True
 
     def close_current_source(self):
         if not self.current_source:
             return
 
-        if self.current_source.close:
-            self.current_source.input.close()
-        elif self.shell.tty:
-            print
-
+        self.current_source.close()
         self.current_source = None
-        self.current_reader = None
 
     def close(self):
         self.close_current_source()
 
     def read_rows(self, max_rows):
-        if not self.current_reader:
+        if not self.current_source:
             return []
 
         rows = []
         for i in xrange(min(max_rows, self.chunk_size)):
             try:
-                row = self.current_reader.next()
+                row = self.current_source.next()
                 self.num_read += 1
 
                 if 0 <= self.max_rows < self.num_read:
@@ -729,13 +833,91 @@ class ImportReader(object):
         return filter(None, rows)
 
 
-class ImportErrors(object):
+class PipeReader(object):
     """
-    A small class for managing import errors
+    A class for reading rows received on a pipe, this is used for reading input from STDIN
+    """
+    def __init__(self, inmsg, options):
+        self.inmsg = inmsg
+        self.chunk_size = options.copy['chunksize']
+        self.header = options.copy['header']
+        self.max_rows = options.copy['maxrows']
+        self.skip_rows = options.copy['skiprows']
+        self.num_read = 0
+        self.exhausted = False
+        self.num_sources = 1
+
+    def start(self):
+        pass
+
+    def read_rows(self, max_rows):
+        rows = []
+        for i in xrange(min(max_rows, self.chunk_size)):
+            row = self.inmsg.recv()
+            if row is None:
+                self.exhausted = True
+                break
+
+            self.num_read += 1
+            if 0 <= self.max_rows < self.num_read:
+                self.exhausted = True
+                break  # max rows exceeded
+
+            if self.header or self.num_read < self.skip_rows:
+                self.header = False  # skip header or initial skip_rows rows
+                continue
+
+            rows.append(row)
+
+        return rows
+
+
+class ImportProcessResult(object):
+    """
+    An object sent from ImportProcess instances to the parent import task in order to indicate progress.
+    """
+    def __init__(self, imported=0):
+        self.imported = imported
+
+
+class FeedingProcessResult(object):
+    """
+    An object sent from FeedingProcess instances to the parent import task in order to indicate progress.
+    """
+    def __init__(self, sent, reader):
+        self.sent = sent
+        self.num_sources = reader.num_sources
+        self.skip_rows = reader.skip_rows
+
+
+class ImportTaskError(object):
+    """
+    An object sent from child processes (feeder or workers) to the parent import task to indicate an error.
+    """
+    def __init__(self, name, msg, rows=None, attempts=1, final=True):
+        self.name = name
+        self.msg = msg
+        self.rows = rows if rows else []
+        self.attempts = attempts
+        self.final = final
+
+    def is_parse_error(self):
+        """
+        We treat read and parse errors as unrecoverable and we have different global counters for giving up when
+        a maximum has been reached. We consider value and type errors as parse errors as well since they
+        are typically non recoverable.
+        """
+        name = self.name
+        return name.startswith('ValueError') or name.startswith('TypeError') or \
+            name.startswith('ParseError') or name.startswith('IndexError') or name.startswith('ReadError')
+
+
+class ImportErrorHandler(object):
+    """
+    A class for managing import errors
     """
     def __init__(self, task):
         self.shell = task.shell
-        self.reader = task.reader
         self.options = task.options
         self.printmsg = task.printmsg
         self.max_attempts = self.options.copy['maxattempts']
@@ -771,42 +953,26 @@ class ImportErrors(object):
             for row in rows:
                 writer.writerow(row)
 
-    def handle_error(self, err, batch):
+    def handle_error(self, err):
         """
         Handle an error by printing the appropriate error message and incrementing the correct counter.
-        Return true if we should retry this batch, false if the error is non-recoverable
         """
         shell = self.shell
-        err = str(err)
 
-        if self.is_parse_error(err):
-            self.parse_errors += len(batch['rows'])
-            self.add_failed_rows(batch['rows'])
-            shell.printerr("Failed to import %d rows: %s -  given up without retries"
-                           % (len(batch['rows']), err))
-            return False
+        if err.is_parse_error():
+            self.parse_errors += len(err.rows)
+            self.add_failed_rows(err.rows)
+            shell.printerr("Failed to import %d rows: %s - %s,  given up without retries"
+                           % (len(err.rows), err.name, err.msg))
         else:
-            self.insert_errors += len(batch['rows'])
-            if batch['attempts'] < self.max_attempts:
-                shell.printerr("Failed to import %d rows: %s -  will retry later, attempt %d of %d"
-                               % (len(batch['rows']), err, batch['attempts'],
-                                  self.max_attempts))
-                return True
+            self.insert_errors += len(err.rows)
+            if not err.final:
+                shell.printerr("Failed to import %d rows: %s - %s,  will retry later, attempt %d of %d"
+                               % (len(err.rows), err.name, err.msg, err.attempts, self.max_attempts))
             else:
-                self.add_failed_rows(batch['rows'])
-                shell.printerr("Failed to import %d rows: %s -  given up after %d attempts"
-                               % (len(batch['rows']), err, batch['attempts']))
-                return False
-
-    @staticmethod
-    def is_parse_error(err):
-        """
-        We treat parse errors as unrecoverable and we have different global counters for giving up when
-        a maximum has been reached. We consider value and type errors as parse errors as well since they
-        are typically non recoverable.
-        """
-        return err.startswith('ValueError') or err.startswith('TypeError') or \
-            err.startswith('ParseError') or err.startswith('IndexError')
+                self.add_failed_rows(err.rows)
+                shell.printerr("Failed to import %d rows: %s - %s,  given up after %d attempts"
+                               % (len(err.rows), err.name, err.msg, err.attempts))
 
 
 class ImportTask(CopyTask):
@@ -818,22 +984,14 @@ class ImportTask(CopyTask):
         CopyTask.__init__(self, shell, ks, table, columns, fname, opts, protocol_version, config_file, 'from')
 
         options = self.options
-        self.ingest_rate = options.copy['ingestrate']
-        self.max_attempts = options.copy['maxattempts']
-        self.header = options.copy['header']
         self.skip_columns = [c.strip() for c in self.options.copy['skipcols'].split(',')]
         self.valid_columns = [c for c in self.columns if c not in self.skip_columns]
         self.table_meta = self.shell.get_table_meta(self.ks, self.table)
-        self.batch_id = 0
         self.receive_meter = RateMeter(log_fcn=self.printmsg,
                                        update_interval=options.copy['reportfrequency'],
                                        log_file=options.copy['ratefile'])
-        self.send_meter = RateMeter(log_fcn=None, update_interval=1)
-        self.reader = ImportReader(self)
-        self.import_errors = ImportErrors(self)
-        self.retries = deque([])
-        self.failed = 0
-        self.succeeded = 0
+        self.error_handler = ImportErrorHandler(self)
+        self.feeding_result = None
         self.sent = 0
 
     def make_params(self):
@@ -861,17 +1019,24 @@ class ImportTask(CopyTask):
         self.printmsg("\nStarting copy of %s.%s with columns %s." % (self.ks, self.table, self.valid_columns))
 
         try:
-            self.reader.start()
             params = self.make_params()
 
-            for i in range(self.num_processes):
-                self.processes.append(ImportProcess(params))
+            for i in range(self.num_processes - 1):
+                self.processes.append(ImportProcess(self.update_params(params, i)))
+
+            feeder = FeedingProcess(self.outmsg.channels[-1], self.inmsg.channels[-1],
+                                    self.outmsg.channels[:-1], self.fname, self.options)
+            self.processes.append(feeder)
 
-            for process in self.processes:
-                process.start()
+            self.start_processes()
+
+            pr = profile_on() if PROFILE_ON else None
 
             self.import_records()
 
+            if pr:
+                profile_off(pr, file_name='parent_profile_%d.txt' % (os.getpid(),))
+
         except Exception, exc:
             shell.printerr(str(exc))
             if shell.debug:
@@ -880,9 +1045,22 @@ class ImportTask(CopyTask):
         finally:
             self.close()
 
-    def close(self):
-        CopyTask.close(self)
-        self.reader.close()
+    def send_stdin_rows(self):
+        """
+        We need to pass stdin rows to the feeder process as it is not safe to pickle or share stdin
+        directly (in case of file the child process would close it). This is a very primitive support
+        for STDIN import in that we we won't start reporting progress until STDIN is fully consumed. I
+        think this is reasonable.
+        """
+        shell = self.shell
+
+        self.printmsg("[Use \. on a line by itself to end input]")
+        for row in shell.use_stdin_reader(prompt='[copy] ', until=r'.'):
+            self.outmsg.channels[-1].send(row)
+
+        self.outmsg.channels[-1].send(None)
+        if shell.tty:
+            print
 
     def import_records(self):
         """
@@ -890,114 +1068,137 @@ class ImportTask(CopyTask):
         Send data (batches or retries) up to the max ingest rate. If we are waiting for stuff to
         receive check the incoming queue.
         """
-        reader = self.reader
-
-        while self.has_more_to_send(reader) or self.has_more_to_receive():
-            if self.has_more_to_send(reader):
-                self.send_batches(reader)
+        if not self.fname:
+            self.send_stdin_rows()
 
-            if self.has_more_to_receive():
-                self.receive()
+        while self.feeding_result is None or self.receive_meter.total_records < self.feeding_result.sent:
+            self.receive_results()
 
-            if self.import_errors.max_exceeded() or not self.all_processes_running():
+            if self.error_handler.max_exceeded() or not self.all_processes_running():
                 break
 
-        if self.import_errors.num_rows_failed:
+        if self.error_handler.num_rows_failed:
             self.shell.printerr("Failed to process %d rows; failed rows written to %s" %
-                                (self.import_errors.num_rows_failed,
-                                 self.import_errors.err_file))
+                                (self.error_handler.num_rows_failed,
+                                 self.error_handler.err_file))
 
         if not self.all_processes_running():
             self.shell.printerr("{} child process(es) died unexpectedly, aborting"
                                 .format(self.num_processes - self.num_live_processes()))
+        else:
+            # it is only safe to write to processes if they are all running because the feeder process
+            # at the moment hangs whilst sending messages to a crashed worker process; in future
+            # we could do something about this by using a BoundedSemaphore to keep track of how many messages are
+            # queued on a pipe
+            for i, _ in enumerate(self.processes):
+                self.outmsg.channels[i].send(None)
+
+            if PROFILE_ON:
+                # allow time for worker processes to write profile results (only works if processes received
+                # the poison pill above)
+                time.sleep(5)
 
         self.printmsg("\n%d rows imported from %d files in %s (%d skipped)." %
                       (self.receive_meter.get_total_records(),
-                       self.reader.num_sources,
+                       self.feeding_result.num_sources if self.feeding_result else 0,
                        self.describe_interval(time.time() - self.time_start),
-                       self.reader.skip_rows))
-
-    def has_more_to_receive(self):
-        return (self.succeeded + self.failed) < self.sent
-
-    def has_more_to_send(self, reader):
-        return (not reader.exhausted) or self.retries
+                       self.feeding_result.skip_rows if self.feeding_result else 0))
 
     def all_processes_running(self):
-        return self.num_live_processes() == self.num_processes
+        return self.num_live_processes() == len(self.processes)
 
-    def receive(self):
-        start_time = time.time()
+    def receive_results(self):
+        """
+        Receive results from the worker processes, which will send the number of rows imported
+        or from the feeder process, which will send the number of rows sent when it has finished sending rows.
+        """
+        aggregate_result = ImportProcessResult()
+        try:
+            for result in self.inmsg.recv(timeout=0.1):
+                if isinstance(result, ImportProcessResult):
+                    aggregate_result.imported += result.imported
+                elif isinstance(result, ImportTaskError):
+                    self.error_handler.handle_error(result)
+                elif isinstance(result, FeedingProcessResult):
+                    self.feeding_result = result
+                else:
+                    raise ValueError("Unexpected result: %s" % (result,))
+        finally:
+            self.receive_meter.increment(aggregate_result.imported)
 
-        while time.time() - start_time < 0.001:
-            try:
-                batch, err = self.inmsg.get(timeout=0.00001)
 
-                if err is None:
-                    self.succeeded += batch['imported']
-                    self.receive_meter.increment(batch['imported'])
-                else:
-                    err = str(err)
+class FeedingProcess(mp.Process):
+    """
+    A process that reads from import sources and sends chunks to worker processes.
+    """
+    def __init__(self, inmsg, outmsg, worker_channels, fname, options):
+        mp.Process.__init__(self, target=self.run)
+        self.inmsg = inmsg
+        self.outmsg = outmsg
+        self.worker_channels = worker_channels
+        self.reader = FilesReader(fname, options) if fname else PipeReader(inmsg, options)
+        self.send_meter = RateMeter(log_fcn=None, update_interval=1)
+        self.ingest_rate = options.copy['ingestrate']
+        self.num_worker_processes = options.copy['numprocesses']
+        self.chunk_id = 0
+
+    def run(self):
+        pr = profile_on() if PROFILE_ON else None
 
-                    if self.import_errors.handle_error(err, batch):
-                        self.retries.append(self.reset_batch(batch))
-                    else:
-                        self.failed += len(batch['rows'])
+        self.inner_run()
 
-            except Queue.Empty:
-                pass
+        if pr:
+            profile_off(pr, file_name='feeder_profile_%d.txt' % (os.getpid(),))
 
-    def send_batches(self, reader):
+    def inner_run(self):
         """
         Send one batch per worker process to the queue unless we have exceeded the ingest rate.
         In the export case we queue everything and let the worker processes throttle using max_requests,
-        here we throttle using the ingest rate in the parent process because of memory usage concerns.
-
-        When we have finished reading the csv file, then send any retries.
+        here we throttle using the ingest rate in the feeding process because of memory usage concerns.
+        When finished we send back to the parent process the total number of rows sent.
         """
-        for _ in xrange(self.num_processes):
-            max_rows = self.ingest_rate - self.send_meter.current_record
-            if max_rows <= 0:
-                self.send_meter.maybe_update()
-                break
+        reader = self.reader
+        reader.start()
+        channels = self.worker_channels
+        sent = 0
+
+        while not reader.exhausted:
+            for ch in channels:
+                try:
+                    max_rows = self.ingest_rate - self.send_meter.current_record
+                    if max_rows <= 0:
+                        self.send_meter.maybe_update(sleep=False)
+                        continue
+
+                    rows = reader.read_rows(max_rows)
+                    if rows:
+                        sent += self.send_chunk(ch, rows)
+                except Exception, exc:
+                    self.outmsg.send(ImportTaskError(exc.__class__.__name__, exc.message))
+
+                if reader.exhausted:
+                    break
 
-            if not reader.exhausted:
-                rows = reader.read_rows(max_rows)
-                if rows:
-                    self.sent += self.send_batch(self.new_batch(rows))
-            elif self.retries:
-                batch = self.retries.popleft()
-                if len(batch['rows']) <= max_rows:
-                    self.send_batch(batch)
-                else:
-                    self.send_batch(self.split_batch(batch, batch['rows'][:max_rows]))
-                    self.retries.append(self.split_batch(batch, batch['rows'][max_rows:]))
-            else:
-                break
+        # send back to the parent process the number of rows sent to the worker processes
+        self.outmsg.send(FeedingProcessResult(sent, reader))
+
+        # wait for poison pill (None)
+        self.inmsg.recv()
 
-    def send_batch(self, batch):
-        batch['attempts'] += 1
-        num_rows = len(batch['rows'])
+    def send_chunk(self, ch, rows):
+        self.chunk_id += 1
+        num_rows = len(rows)
         self.send_meter.increment(num_rows)
-        self.outmsg.put(batch)
+        ch.send({'id': self.chunk_id, 'rows': rows, 'imported': 0, 'num_rows_sent': num_rows})
         return num_rows
 
-    def new_batch(self, rows):
-        self.batch_id += 1
-        return self.make_batch(self.batch_id, rows, 0)
-
-    @staticmethod
-    def reset_batch(batch):
-        batch['imported'] = 0
-        return batch
-
-    @staticmethod
-    def split_batch(batch, rows):
-        return ImportTask.make_batch(batch['id'], rows, batch['attempts'])
+    def close(self):
+        self.reader.close()
+        self.inmsg.close()
+        self.outmsg.close()
 
-    @staticmethod
-    def make_batch(batch_id, rows, attempts):
-        return {'id': batch_id, 'rows': rows, 'attempts': attempts, 'imported': 0}
+        for ch in self.worker_channels:
+            ch.close()
 
 
 class ChildProcess(mp.Process):
@@ -1029,6 +1230,7 @@ class ChildProcess(mp.Process):
         self.decimal_sep = options.copy['decimalsep']
         self.thousands_sep = options.copy['thousandssep']
         self.boolean_styles = options.copy['boolstyle']
+        self.max_attempts = options.copy['maxattempts']
         # Here we inject some failures for testing purposes, only if this environment variable is set
         if os.environ.get('CQLSH_COPY_TEST_FAILURES', ''):
             self.test_failures = json.loads(os.environ.get('CQLSH_COPY_TEST_FAILURES', ''))
@@ -1144,7 +1346,6 @@ class ExportProcess(ChildProcess):
         self.encoding = options.copy['encoding']
         self.float_precision = options.copy['float_precision']
         self.nullval = options.copy['nullval']
-        self.max_attempts = options.copy['maxattempts']
         self.max_requests = options.copy['maxrequests']
 
         self.hosts_to_sessions = dict()
@@ -1172,7 +1373,7 @@ class ExportProcess(ChildProcess):
                 time.sleep(0.001)  # 1 millisecond
                 continue
 
-            token_range, info = self.inmsg.get()
+            token_range, info = self.inmsg.recv()
             self.start_request(token_range, info)
 
     @staticmethod
@@ -1190,7 +1391,7 @@ class ExportProcess(ChildProcess):
     def report_error(self, err, token_range=None):
         msg = self.get_error_message(err, print_traceback=self.debug)
         self.printdebugmsg(msg)
-        self.outmsg.put((token_range, Exception(msg)))
+        self.outmsg.send((token_range, Exception(msg)))
 
     def start_request(self, token_range, info):
         """
@@ -1253,7 +1454,8 @@ class ExportProcess(ChildProcess):
             default_retry_policy=ExpBackoffRetryPolicy(self),
             compression=None,
             control_connection_timeout=self.connect_timeout,
-            connect_timeout=self.connect_timeout)
+            connect_timeout=self.connect_timeout,
+            idle_heartbeat_interval=0)
         session = ExportSession(new_cluster, self)
         self.hosts_to_sessions[host] = session
         return session
@@ -1265,7 +1467,7 @@ class ExportProcess(ChildProcess):
                 self.write_rows_to_csv(token_range, rows)
             else:
                 self.write_rows_to_csv(token_range, rows)
-                self.outmsg.put((None, None))
+                self.outmsg.send((None, None))
                 session.complete_request()
 
         def err_callback(err):
@@ -1286,7 +1488,7 @@ class ExportProcess(ChildProcess):
                 writer.writerow(map(self.format_value, row))
 
             data = (output.getvalue(), len(rows))
-            self.outmsg.put((token_range, data))
+            self.outmsg.send((token_range, data))
             output.close()
 
         except Exception, e:
@@ -1376,7 +1578,7 @@ class ImportConversion(object):
     A class for converting strings to values when importing from csv, used by ImportProcess,
     the parent.
     """
-    def __init__(self, parent, table_meta, statement):
+    def __init__(self, parent, table_meta, statement=None):
         self.ks = parent.ks
         self.table = parent.table
         self.columns = parent.valid_columns
@@ -1391,9 +1593,37 @@ class ImportConversion(object):
         self.primary_key_indexes = [self.columns.index(col.name) for col in self.table_meta.primary_key]
         self.partition_key_indexes = [self.columns.index(col.name) for col in self.table_meta.partition_key]
 
+        if statement is None:
+            self.use_prepared_statements = False
+            statement = self._get_primary_key_statement(parent, table_meta)
+        else:
+            self.use_prepared_statements = True
+
         self.proto_version = statement.protocol_version
-        self.cqltypes = dict([(c.name, c.type) for c in statement.column_metadata])
-        self.converters = dict([(c.name, self._get_converter(c.type)) for c in statement.column_metadata])
+
+        # the cql types and converters for the prepared statement, either the full statement or only the primary keys
+        self.cqltypes = [c.type for c in statement.column_metadata]
+        self.converters = [self._get_converter(c.type) for c in statement.column_metadata]
+
+        # the cql types for the entire statement, these are the same as the types above but
+        # only when using prepared statements
+        self.coltypes = [table_meta.columns[name].typestring for name in parent.valid_columns]
+        # these functions are used for non-prepared statements to protect values with quotes if required
+        self.protectors = [protect_value if t in ('ascii', 'text', 'timestamp', 'date', 'time', 'inet') else lambda v: v
+                           for t in self.coltypes]
+
+    @staticmethod
+    def _get_primary_key_statement(parent, table_meta):
+        """
+        We prepare a query statement to find out the types of the partition key columns so we can
+        route the update query to the correct replicas. As far as I understood this is the easiest
+        way to find out the types of the partition columns, we will never use this prepared statement
+        """
+        where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
+        select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(parent.ks),
+                                                         protect_name(parent.table),
+                                                         where_clause)
+        return parent.session.prepare(select_query)
 
     def _get_converter(self, cql_type):
         """
@@ -1581,27 +1811,25 @@ class ImportConversion(object):
 
         return converters.get(cql_type.typename, convert_unknown)
 
-    def get_row_values(self, row):
+    def convert_row(self, row):
         """
-        Parse the row into a list of row values to be returned
+        Convert the row into a list of parsed values if using prepared statements, else simply apply the
+        protection functions to escape values with quotes when required. Also check on the row length and
+        make sure primary partition key values aren't missing.
         """
-        def convert(n, val):
-            try:
-                return self.converters[self.columns[n]](val)
-            except Exception, e:
-                raise ParseError(e.message)
+        converters = self.converters if self.use_prepared_statements else self.protectors
 
-        ret = [None] * len(row)
-        for i, val in enumerate(row):
-            if val != self.nullval:
-                ret[i] = convert(i, val)
-            else:
-                if i in self.primary_key_indexes:
-                    raise ParseError(self.get_null_primary_key_message(i))
+        if len(row) != len(converters):
+            raise ParseError('Invalid row length %d should be %d' % (len(row), len(converters)))
 
-                ret[i] = None
+        for i in self.primary_key_indexes:
+            if row[i] == self.nullval:
+                raise ParseError(self.get_null_primary_key_message(i))
 
-        return ret
+        try:
+            return [conv(val) for conv, val in zip(converters, row)]
+        except Exception, e:
+            raise ParseError(e.message)
 
     def get_null_primary_key_message(self, idx):
         message = "Cannot insert null value for primary key column '%s'." % (self.columns[idx],)
@@ -1610,31 +1838,111 @@ class ImportConversion(object):
                        " the WITH NULL=<marker> option for COPY."
         return message
 
-    def get_row_partition_key_values(self, row):
+    def get_row_partition_key_values_fcn(self):
         """
-        Return a string composed of the partition key values, serialized and binary packed -
-        as expected by metadata.get_replicas(), see also BoundStatement.routing_key.
+        Return a function to convert a row into a string composed of the partition key values serialized
+        and binary packed (the tokens on the ring). Depending on whether we are using prepared statements, we
+        may have to convert the primary key values first, so we have two different serialize_value implementations.
+        We also return different functions depending on how many partition key indexes we have (single or multiple).
+        See also BoundStatement.routing_key.
         """
-        def serialize(n):
-            try:
-                c, v = self.columns[n], row[n]
-                if v == self.nullval:
-                    raise ParseError(self.get_null_primary_key_message(n))
-                return self.cqltypes[c].serialize(self.converters[c](v), self.proto_version)
-            except Exception, e:
-                raise ParseError(e.message)
+        def serialize_value_prepared(n, v):
+            return self.cqltypes[n].serialize(v, self.proto_version)
+
+        def serialize_value_not_prepared(n, v):
+            return self.cqltypes[n].serialize(self.converters[n](v), self.proto_version)
 
         partition_key_indexes = self.partition_key_indexes
-        if len(partition_key_indexes) == 1:
-            return serialize(partition_key_indexes[0])
-        else:
+        serialize = serialize_value_prepared if self.use_prepared_statements else serialize_value_not_prepared
+
+        def serialize_row_single(row):
+            return serialize(partition_key_indexes[0], row[partition_key_indexes[0]])
+
+        def serialize_row_multiple(row):
             pk_values = []
             for i in partition_key_indexes:
-                val = serialize(i)
+                val = serialize(i, row[i])
                 l = len(val)
                 pk_values.append(struct.pack(">H%dsB" % l, l, val, 0))
             return b"".join(pk_values)
 
+        if len(partition_key_indexes) == 1:
+            return serialize_row_single
+        return serialize_row_multiple
+
+
+class TokenMap(object):
+    """
+    A wrapper around the metadata token map to speed things up by caching ring token *values* and
+    replicas. It is very important that we use the token values, which are primitive types, rather
+    than the tokens classes when calling bisect_right() in split_batches(). If we use primitive values,
+    the bisect is done in compiled code whilst with token classes each comparison requires a call
+    into the interpreter to perform the cmp operation defined in Python. A simple test with 1 million bisect
+    operations on an array of 2048 tokens was done in 0.37 seconds with primitives and 2.25 seconds with
+    token classes. This is significant for large datasets because we need to do a bisect for each single row,
+    and if VNODES are used, the size of the token map can get quite large too.
+    """
+    def __init__(self, ks, hostname, local_dc, session):
+
+        self.ks = ks
+        self.hostname = hostname
+        self.local_dc = local_dc
+        self.metadata = session.cluster.metadata
+
+        self._initialize_ring()
+
+        # Note that refresh metadata is disabled by default and we currenlty do not intercept it
+        # If hosts are added, removed or moved during a COPY operation our token map is no longer optimal
+        # However we can cope with hosts going down and up since we filter for replicas that are up when
+        # making each batch
+
+    def _initialize_ring(self):
+        token_map = self.metadata.token_map
+        if token_map is None:
+            self.ring = [0]
+            self.replicas = [(self.metadata.get_host(self.hostname),)]
+            self.pk_to_token_value = lambda pk: 0
+            return
+
+        token_map.rebuild_keyspace(self.ks, build_if_absent=True)
+        tokens_to_hosts = token_map.tokens_to_hosts_by_ks.get(self.ks, None)
+        from_key = token_map.token_class.from_key
+
+        self.ring = [token.value for token in token_map.ring]
+        self.replicas = [tuple(tokens_to_hosts[token]) for token in token_map.ring]
+        self.pk_to_token_value = lambda pk: from_key(pk).value
+
+    @staticmethod
+    def get_ring_pos(ring, val):
+        idx = bisect_right(ring, val)
+        return idx if idx < len(ring) else 0
+
+    def filter_replicas(self, hosts):
+        shuffled = tuple(sorted(hosts, key=lambda k: random.random()))
+        return filter(lambda r: r.is_up and r.datacenter == self.local_dc, shuffled) if hosts else ()
+
+
+class FastTokenAwarePolicy(DCAwareRoundRobinPolicy):
+    """
+    Send to any replicas attached to the query, or else fall back to DCAwareRoundRobinPolicy
+    """
+
+    def __init__(self, local_dc='', used_hosts_per_remote_dc=0):
+        DCAwareRoundRobinPolicy.__init__(self, local_dc, used_hosts_per_remote_dc)
+
+    def make_query_plan(self, working_keyspace=None, query=None):
+        """
+        Extend TokenAwarePolicy.make_query_plan() so that we choose the same replicas in preference
+        and most importantly we avoid repeating the (slow) bisect
+        """
+        replicas = query.replicas if hasattr(query, 'replicas') else []
+        for r in replicas:
+            yield r
+
+        for r in DCAwareRoundRobinPolicy.make_query_plan(self, working_keyspace, query):
+            if r not in replicas:
+                yield r
+
 
 class ImportProcess(ChildProcess):
 
@@ -1650,7 +1958,12 @@ class ImportProcess(ChildProcess):
         self.max_attempts = options.copy['maxattempts']
         self.min_batch_size = options.copy['minbatchsize']
         self.max_batch_size = options.copy['maxbatchsize']
+        self.use_prepared_statements = options.copy['preparedstatements']
+        self.dialect_options = options.dialect
         self._session = None
+        self.query = None
+        self.conv = None
+        self.make_statement = None
 
     @property
     def session(self):
@@ -1661,12 +1974,13 @@ class ImportProcess(ChildProcess):
                 cql_version=self.cql_version,
                 protocol_version=self.protocol_version,
                 auth_provider=self.auth_provider,
-                load_balancing_policy=TokenAwarePolicy(DCAwareRoundRobinPolicy(local_dc=self.local_dc)),
+                load_balancing_policy=FastTokenAwarePolicy(local_dc=self.local_dc),
                 ssl_options=ssl_settings(self.hostname, self.config_file) if self.ssl else None,
                 default_retry_policy=ExpBackoffRetryPolicy(self),
                 compression=None,
                 control_connection_timeout=self.connect_timeout,
-                connect_timeout=self.connect_timeout)
+                connect_timeout=self.connect_timeout,
+                idle_heartbeat_interval=0)
 
             self._session = cluster.connect(self.ks)
             self._session.default_timeout = None
@@ -1674,13 +1988,12 @@ class ImportProcess(ChildProcess):
 
     def run(self):
         try:
-            table_meta = self.session.cluster.metadata.keyspaces[self.ks].tables[self.table]
-            is_counter = ("counter" in [table_meta.columns[name].typestring for name in self.valid_columns])
+            pr = profile_on() if PROFILE_ON else None
 
-            if is_counter:
-                self.run_counter(table_meta)
-            else:
-                self.run_normal(table_meta)
+            self.inner_run(*self.make_params())
+
+            if pr:
+                profile_off(pr, file_name='worker_profile_%d.txt' % (os.getpid(),))
 
         except Exception, exc:
             if self.debug:
@@ -1694,67 +2007,88 @@ class ImportProcess(ChildProcess):
             self._session.cluster.shutdown()
         ChildProcess.close(self)
 
-    def run_counter(self, table_meta):
-        """
-        Main run method for tables that contain counter columns.
-        """
-        query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
-
-        # We prepare a query statement to find out the types of the partition key columns so we can
-        # route the update query to the correct replicas. As far as I understood this is the easiest
-        # way to find out the types of the partition columns, we will never use this prepared statement
-        where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
-        select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(self.ks), protect_name(self.table), where_clause)
-        conv = ImportConversion(self, table_meta, self.session.prepare(select_query))
-
-        while True:
-            batch = self.inmsg.get()
-            try:
-                for b in self.split_batches(batch, conv):
-                    self.send_counter_batch(query, conv, b)
+    def make_params(self):
+        metadata = self.session.cluster.metadata
+        table_meta = metadata.keyspaces[self.ks].tables[self.table]
+
+        prepared_statement = None
+        is_counter = ("counter" in [table_meta.columns[name].typestring for name in self.valid_columns])
+        if is_counter:
+            query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
+            make_statement = self.wrap_make_statement(self.make_counter_batch_statement)
+        elif self.use_prepared_statements:
+            query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
+                                                            protect_name(self.table),
+                                                            ', '.join(protect_names(self.valid_columns),),
+                                                            ', '.join(['?' for _ in self.valid_columns]))
+
+            query = self.session.prepare(query)
+            query.consistency_level = self.consistency_level
+            prepared_statement = query
+            make_statement = self.wrap_make_statement(self.make_prepared_batch_statement)
+        else:
+            query = 'INSERT INTO %s.%s (%s) VALUES (%%s)' % (protect_name(self.ks),
+                                                             protect_name(self.table),
+                                                             ', '.join(protect_names(self.valid_columns),))
+            make_statement = self.wrap_make_statement(self.make_non_prepared_batch_statement)
 
-            except Exception, exc:
-                self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                if self.debug:
-                    traceback.print_exc(exc)
+        conv = ImportConversion(self, table_meta, prepared_statement)
+        tm = TokenMap(self.ks, self.hostname, self.local_dc, self.session)
+        return query, conv, tm, make_statement
 
-    def run_normal(self, table_meta):
+    def inner_run(self, query, conv, tm, make_statement):
         """
-        Main run method for normal tables, i.e. tables that do not contain counter columns.
+        Main run method. Note that we bind self methods that are called inside loops
+        for performance reasons.
         """
-        query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
-                                                        protect_name(self.table),
-                                                        ', '.join(protect_names(self.valid_columns),),
-                                                        ', '.join(['?' for _ in self.valid_columns]))
+        self.query = query
+        self.conv = conv
+        self.make_statement = make_statement
 
-        query_statement = self.session.prepare(query)
-        query_statement.consistency_level = self.consistency_level
-        conv = ImportConversion(self, table_meta, query_statement)
+        convert_rows = self.convert_rows
+        split_into_batches = self.split_into_batches
+        result_callback = self.result_callback
+        err_callback = self.err_callback
+        session = self.session
 
         while True:
-            batch = self.inmsg.get()
+            chunk = self.inmsg.recv()
+            if chunk is None:
+                break
+
             try:
-                for b in self.split_batches(batch, conv):
-                    self.send_normal_batch(conv, query_statement, b)
+                chunk['rows'] = convert_rows(conv, chunk)
+                for replicas, batch in split_into_batches(chunk, conv, tm):
+                    statement = make_statement(query, conv, chunk, batch, replicas)
+                    future = session.execute_async(statement)
+                    future.add_callbacks(callback=result_callback, callback_args=(batch, chunk),
+                                         errback=err_callback, errback_args=(batch, chunk, replicas))
 
             except Exception, exc:
-                self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                if self.debug:
-                    traceback.print_exc(exc)
+                self.report_error(exc, chunk, chunk['rows'])
 
-    def send_counter_batch(self, query_text, conv, batch):
-        if self.test_failures and self.maybe_inject_failures(batch):
-            return
+    def wrap_make_statement(self, inner_make_statement):
+        def make_statement(query, conv, chunk, batch, replicas):
+            try:
+                return inner_make_statement(query, conv, batch, replicas)
+            except Exception, exc:
+                print "Failed to make batch statement: {}".format(exc)
+                self.report_error(exc, chunk, batch['rows'])
+                return None
 
-        error_rows = []
-        batch_statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+        def make_statement_with_failures(query, conv, chunk, batch, replicas):
+            failed_batch = self.maybe_inject_failures(batch)
+            if failed_batch:
+                return failed_batch
+            return make_statement(query, conv, chunk, batch, replicas)
 
-        for r in batch['rows']:
-            row = self.filter_row_values(r)
-            if len(row) != len(self.valid_columns):
-                error_rows.append(row)
-                continue
+        return make_statement_with_failures if self.test_failures else make_statement
 
+    def make_counter_batch_statement(self, query, conv, batch, replicas):
+        statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+        statement.replicas = replicas
+        statement.keyspace = self.ks
+        for row in batch['rows']:
             where_clause = []
             set_clause = []
             for i, value in enumerate(row):
@@ -1763,65 +2097,61 @@ class ImportProcess(ChildProcess):
                 else:
                     set_clause.append("%s=%s+%s" % (self.valid_columns[i], self.valid_columns[i], value))
 
-            full_query_text = query_text % (','.join(set_clause), ' AND '.join(where_clause))
-            batch_statement.add(full_query_text)
-
-        self.execute_statement(batch_statement, batch)
+            full_query_text = query % (','.join(set_clause), ' AND '.join(where_clause))
+            statement.add(full_query_text)
+        return statement
 
-        if error_rows:
-            self.outmsg.put((ImportTask.split_batch(batch, error_rows),
-                            '%s - %s' % (ParseError.__name__, "Failed to parse one or more rows")))
+    def make_prepared_batch_statement(self, query, _, batch, replicas):
+        """
+        Return a batch statement. This is an optimized version of:
 
-    def send_normal_batch(self, conv, query_statement, batch):
-        if self.test_failures and self.maybe_inject_failures(batch):
-            return
+            statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+            for row in batch['rows']:
+                statement.add(query, row)
 
-        good_rows, converted_rows, errors = self.convert_rows(conv, batch['rows'])
+        We could optimize further by removing bound_statements altogether but we'd have to duplicate much
+        more driver's code (BoundStatement.bind()).
+        """
+        statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+        statement.replicas = replicas
+        statement.keyspace = self.ks
+        statement._statements_and_parameters = [(True, query.query_id, query.bind(r).values) for r in batch['rows']]
+        return statement
 
-        if converted_rows:
-            try:
-                statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
-                for row in converted_rows:
-                    statement.add(query_statement, row)
-                self.execute_statement(statement, ImportTask.split_batch(batch, good_rows))
-            except Exception, exc:
-                self.err_callback(exc, ImportTask.split_batch(batch, good_rows))
+    def make_non_prepared_batch_statement(self, query, _, batch, replicas):
+        statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+        statement.replicas = replicas
+        statement.keyspace = self.ks
+        statement._statements_and_parameters = [(False, query % (','.join(r),), ()) for r in batch['rows']]
+        return statement
 
-        if errors:
-            for msg, rows in errors.iteritems():
-                self.outmsg.put((ImportTask.split_batch(batch, rows),
-                                '%s - %s' % (ParseError.__name__, msg)))
-
-    def convert_rows(self, conv, rows):
+    def convert_rows(self, conv, chunk):
         """
-        Try to convert each row. If conversion is OK then add the converted result to converted_rows
-        and the original string to good_rows. Else add the original string to error_rows. Return the three
-        arrays.
+        Return converted rows and report any errors during conversion.
         """
-        good_rows = []
-        errors = defaultdict(list)
-        converted_rows = []
+        def filter_row_values(row):
+            return [v for i, v in enumerate(row) if i not in self.skip_column_indexes]
 
-        for r in rows:
-            row = self.filter_row_values(r)
-            if len(row) != len(self.valid_columns):
-                msg = 'Invalid row length %d should be %d' % (len(row), len(self.valid_columns))
-                errors[msg].append(row)
-                continue
+        if self.skip_column_indexes:
+            rows = [filter_row_values(r) for r in list(csv.reader(chunk['rows'], **self.dialect_options))]
+        else:
+            rows = list(csv.reader(chunk['rows'], **self.dialect_options))
 
-            try:
-                converted_rows.append(conv.get_row_values(row))
-                good_rows.append(row)
-            except ParseError, err:
-                errors[err.message].append(row)
+        errors = defaultdict(list)
 
-        return good_rows, converted_rows, errors
+        def convert_row(r):
+            try:
+                return conv.convert_row(r)
+            except Exception, err:
+                errors[err.message].append(r)
+                return None
 
-    def filter_row_values(self, row):
-        if not self.skip_column_indexes:
-            return row
+        converted_rows = filter(None, [convert_row(r) for r in rows])
 
-        return [v for i, v in enumerate(row) if i not in self.skip_column_indexes]
+        if errors:
+            for msg, rows in errors.iteritems():
+                self.report_error(ParseError(msg), chunk, rows)
+        return converted_rows
 
     def maybe_inject_failures(self, batch):
         """
@@ -1836,86 +2166,94 @@ class ImportProcess(ChildProcess):
                 if batch['attempts'] < failing_batch['failures']:
                     statement = SimpleStatement("INSERT INTO badtable (a, b) VALUES (1, 2)",
                                                 consistency_level=self.consistency_level)
-                    self.execute_statement(statement, batch)
-                    return True
+                    return statement
 
         if 'exit_batch' in self.test_failures:
             exit_batch = self.test_failures['exit_batch']
             if exit_batch['id'] == batch['id']:
                 sys.exit(1)
 
-        return False  # carry on as normal
+        return None  # carry on as normal
 
-    def execute_statement(self, statement, batch):
-        future = self.session.execute_async(statement)
-        future.add_callbacks(callback=self.result_callback, callback_args=(batch, ),
-                             errback=self.err_callback, errback_args=(batch, ))
+    @staticmethod
+    def make_batch(batch_id, rows, attempts=1):
+        return {'id': batch_id, 'rows': rows, 'attempts': attempts}
 
-    def split_batches(self, batch, conv):
+    def split_into_batches(self, chunk, conv, tm):
         """
-        Batch rows by partition key, if there are at least min_batch_size (2)
-        rows with the same partition key. These batches can be as big as they want
-        since this translates to a single insert operation server side.
-
-        If there are less than min_batch_size rows for a partition, work out the
-        first replica for this partition and add the rows to replica left-over rows.
-
-        Then batch the left-overs of each replica up to max_batch_size.
+        Batch rows by ring position or replica.
+        If there are at least min_batch_size rows for a ring position then split these rows into
+        groups of max_batch_size and send a batch for each group, using all replicas for this ring position.
+        Otherwise, we are forced to batch by replica, and here unfortunately we can only choose one replica to
+        guarantee common replicas across partition keys. We are typically able
+        to batch by ring position for small clusters or when VNODES are not used. For large clusters with VNODES
+        it may not be possible, in this case it helps to increase the CHUNK SIZE but up to a limit, otherwise
+        we may choke the cluster.
         """
-        rows_by_pk = defaultdict(list)
+
+        rows_by_ring_pos = defaultdict(list)
         errors = defaultdict(list)
 
-        for row in batch['rows']:
+        min_batch_size = self.min_batch_size
+        max_batch_size = self.max_batch_size
+        ring = tm.ring
+
+        get_row_partition_key_values = conv.get_row_partition_key_values_fcn()
+        pk_to_token_value = tm.pk_to_token_value
+        get_ring_pos = tm.get_ring_pos
+        make_batch = self.make_batch
+
+        for row in chunk['rows']:
             try:
-                pk = conv.get_row_partition_key_values(row)
-                rows_by_pk[pk].append(row)
-            except ParseError, e:
+                pk = get_row_partition_key_values(row)
+                rows_by_ring_pos[get_ring_pos(ring, pk_to_token_value(pk))].append(row)
+            except Exception, e:
                 errors[e.message].append(row)
 
         if errors:
             for msg, rows in errors.iteritems():
-                self.outmsg.put((ImportTask.split_batch(batch, rows),
-                                 '%s - %s' % (ParseError.__name__, msg)))
+                self.report_error(ParseError(msg), chunk, rows)
 
+        replicas = tm.replicas
+        filter_replicas = tm.filter_replicas
         rows_by_replica = defaultdict(list)
-        for pk, rows in rows_by_pk.iteritems():
-            if len(rows) >= self.min_batch_size:
-                yield ImportTask.make_batch(batch['id'], rows, batch['attempts'])
+        for ring_pos, rows in rows_by_ring_pos.iteritems():
+            if len(rows) > min_batch_size:
+                for i in xrange(0, len(rows), max_batch_size):
+                    yield filter_replicas(replicas[ring_pos]), make_batch(chunk['id'], rows[i:i + max_batch_size])
             else:
-                replica = self.get_replica(pk)
-                rows_by_replica[replica].extend(rows)
-
-        for replica, rows in rows_by_replica.iteritems():
-            for b in self.batches(rows, batch):
-                yield b
-
-    def get_replica(self, pk):
-        """
-        Return the first replica or the host we are already connected to if there are no local
-        replicas that are up. We always use the first replica to match the replica chosen by the driver
-        TAR, see TokenAwarePolicy.make_query_plan().
-        """
-        metadata = self.session.cluster.metadata
-        replicas = filter(lambda r: r.is_up and r.datacenter == self.local_dc, metadata.get_replicas(self.ks, pk))
-        ret = replicas[0].address if len(replicas) > 0 else self.hostname
-        return ret
-
-    def batches(self, rows, batch):
-        """
-        Split rows into batches of max_batch_size
-        """
-        for i in xrange(0, len(rows), self.max_batch_size):
-            yield ImportTask.make_batch(batch['id'], rows[i:i + self.max_batch_size], batch['attempts'])
-
-    def result_callback(self, _, batch):
-        batch['imported'] = len(batch['rows'])
-        batch['rows'] = []  # no need to resend these, just send the count in 'imported'
-        self.outmsg.put((batch, None))
-
-    def err_callback(self, response, batch):
-        self.outmsg.put((batch, '%s - %s' % (response.__class__.__name__, response.message)))
+                # select only the first valid replica to guarantee more overlap or none at all
+                rows_by_replica[filter_replicas(replicas[ring_pos])[:1]].extend(rows)
+
+        # Now send the batches by replica
+        for replicas, rows in rows_by_replica.iteritems():
+            for i in xrange(0, len(rows), max_batch_size):
+                yield replicas, make_batch(chunk['id'], rows[i:i + max_batch_size])
+
+    def result_callback(self, _, batch, chunk):
+        self.update_chunk(batch['rows'], chunk)
+
+    def err_callback(self, response, batch, chunk, replicas):
+        err_is_final = batch['attempts'] >= self.max_attempts
+        self.report_error(response, chunk, batch['rows'], batch['attempts'], err_is_final)
+        if not err_is_final:
+            batch['attempts'] += 1
+            statement = self.make_statement(self.query, self.conv, chunk, batch, replicas)
+            future = self.session.execute_async(statement)
+            future.add_callbacks(callback=self.result_callback, callback_args=(batch, chunk),
+                                 errback=self.err_callback, errback_args=(batch, chunk, replicas))
+
+    def report_error(self, err, chunk, rows=None, attempts=1, final=True):
         if self.debug:
-            traceback.print_exc(response)
+            traceback.print_exc(err)
+        self.outmsg.send(ImportTaskError(err.__class__.__name__, err.message, rows, attempts, final))
+        if final:
+            self.update_chunk(rows, chunk)
+
+    def update_chunk(self, rows, chunk):
+        chunk['imported'] += len(rows)
+        if chunk['imported'] == chunk['num_rows_sent']:
+            self.outmsg.send(ImportProcessResult(chunk['num_rows_sent']))
 
 
 class RateMeter(object):
@@ -1937,11 +2275,19 @@ class RateMeter(object):
         self.current_record += n
         self.maybe_update()
 
-    def maybe_update(self):
+    def maybe_update(self, sleep=False):
+        if self.current_record == 0:
+            return
+
         new_checkpoint_time = time.time()
-        if new_checkpoint_time - self.last_checkpoint_time >= self.update_interval:
+        time_difference = new_checkpoint_time - self.last_checkpoint_time
+        if time_difference >= self.update_interval:
             self.update(new_checkpoint_time)
             self.log_message()
+        elif sleep:
+            remaining_time = time_difference - self.update_interval
+            if remaining_time > 0.000001:
+                time.sleep(remaining_time)
 
     def update(self, new_checkpoint_time):
         time_difference = new_checkpoint_time - self.last_checkpoint_time

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/pylib/cqlshlib/util.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/util.py b/pylib/cqlshlib/util.py
index 281aad6..3ee128d 100644
--- a/pylib/cqlshlib/util.py
+++ b/pylib/cqlshlib/util.py
@@ -23,6 +23,12 @@ from itertools import izip
 from datetime import timedelta, tzinfo
 from StringIO import StringIO
 
+try:
+    from line_profiler import LineProfiler
+    HAS_LINE_PROFILER = True
+except ImportError:
+    HAS_LINE_PROFILER = False
+
 ZERO = timedelta(0)
 
 
@@ -126,18 +132,35 @@ def get_file_encoding_bomsize(filename):
     else:
         file_encoding, size = "utf-8", 0
 
-    return (file_encoding, size)
+    return file_encoding, size
+
 
+def profile_on(fcn_names=None):
+    if fcn_names and HAS_LINE_PROFILER:
+        pr = LineProfiler()
+        for fcn_name in fcn_names:
+            pr.add_function(fcn_name)
+        pr.enable()
+        return pr
 
-def profile_on():
     pr = cProfile.Profile()
     pr.enable()
     return pr
 
 
-def profile_off(pr):
+def profile_off(pr, file_name):
     pr.disable()
     s = StringIO()
-    ps = pstats.Stats(pr, stream=s).sort_stats('cumulative')
-    ps.print_stats()
-    print s.getvalue()
+
+    if HAS_LINE_PROFILER and isinstance(pr, LineProfiler):
+        pr.print_stats(s)
+    else:
+        ps = pstats.Stats(pr, stream=s).sort_stats('cumulative')
+        ps.print_stats()
+
+    ret = s.getvalue()
+    if file_name:
+        with open(file_name, 'w') as f:
+            print "Writing to %s\n" % (f.name, )
+            f.write(ret)
+    return ret

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/pylib/setup.py
----------------------------------------------------------------------
diff --git a/pylib/setup.py b/pylib/setup.py
index 704d077..3654502 100755
--- a/pylib/setup.py
+++ b/pylib/setup.py
@@ -16,9 +16,11 @@
 # limitations under the License.
 
 from distutils.core import setup
+from Cython.Build import cythonize
 
 setup(
     name="cassandra-pylib",
     description="Cassandra Python Libraries",
     packages=["cqlshlib"],
+    ext_modules=cythonize("cqlshlib/copyutil.py"),
 )


[22/23] cassandra git commit: Merge commit '49c616cf0fc9c8d4649e2ec71a07fb9fd7831318' into cassandra-3.5

Posted by sl...@apache.org.
Merge commit '49c616cf0fc9c8d4649e2ec71a07fb9fd7831318' into cassandra-3.5


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6329d54a
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6329d54a
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6329d54a

Branch: refs/heads/trunk
Commit: 6329d54a6c802e703902ee6cf842e983820d144b
Parents: c9e9b62 49c616c
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue Mar 8 10:25:32 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Mar 8 10:25:54 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                |    1 +
 bin/cqlsh.py               |   33 +-
 pylib/cqlshlib/copyutil.py | 1173 +++++++++++++++++++++++++--------------
 pylib/cqlshlib/util.py     |   35 +-
 pylib/setup.py             |    2 +
 5 files changed, 824 insertions(+), 420 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6329d54a/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index c791139,5b92143..c88ae54
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -7,42 -7,11 +7,43 @@@ Merged from 2.2
   * Only log yaml config once, at startup (CASSANDRA-11217)
   * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
  Merged from 2.1:
 - * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
+  * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
 + * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
  
 -3.0.4
 - * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
 +
 +3.4
 + * (cqlsh) add cqlshrc option to always connect using ssl (CASSANDRA-10458)
 + * Cleanup a few resource warnings (CASSANDRA-11085)
 + * Allow custom tracing implementations (CASSANDRA-10392)
 + * Extract LoaderOptions to be able to be used from outside (CASSANDRA-10637)
 + * fix OnDiskIndexTest to properly treat empty ranges (CASSANDRA-11205)
 + * fix TrackerTest to handle new notifications (CASSANDRA-11178)
 + * add SASI validation for partitioner and complex columns (CASSANDRA-11169)
 + * Add caching of encrypted credentials in PasswordAuthenticator (CASSANDRA-7715)
 + * fix SASI memtable switching on flush (CASSANDRA-11159)
 + * Remove duplicate offline compaction tracking (CASSANDRA-11148)
 + * fix EQ semantics of analyzed SASI indexes (CASSANDRA-11130)
 + * Support long name output for nodetool commands (CASSANDRA-7950)
 + * Encrypted hints (CASSANDRA-11040)
 + * SASI index options validation (CASSANDRA-11136)
 + * Optimize disk seek using min/max column name meta data when the LIMIT clause is used
 +   (CASSANDRA-8180)
 + * Add LIKE support to CQL3 (CASSANDRA-11067)
 + * Generic Java UDF types (CASSANDRA-10819)
 + * cqlsh: Include sub-second precision in timestamps by default (CASSANDRA-10428)
 + * Set javac encoding to utf-8 (CASSANDRA-11077)
 + * Integrate SASI index into Cassandra (CASSANDRA-10661)
 + * Add --skip-flush option to nodetool snapshot
 + * Skip values for non-queried columns (CASSANDRA-10657)
 + * Add support for secondary indexes on static columns (CASSANDRA-8103)
 + * CommitLogUpgradeTestMaker creates broken commit logs (CASSANDRA-11051)
 + * Add metric for number of dropped mutations (CASSANDRA-10866)
 + * Simplify row cache invalidation code (CASSANDRA-10396)
 + * Support user-defined compaction through nodetool (CASSANDRA-10660)
 + * Stripe view locks by key and table ID to reduce contention (CASSANDRA-10981)
 + * Add nodetool gettimeout and settimeout commands (CASSANDRA-10953)
 + * Add 3.0 metadata to sstablemetadata output (CASSANDRA-10838)
 +Merged from 3.0:
   * MV should only query complex columns included in the view (CASSANDRA-11069)
   * Failed aggregate creation breaks server permanently (CASSANDRA-11064)
   * Add sstabledump tool (CASSANDRA-7464)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6329d54a/bin/cqlsh.py
----------------------------------------------------------------------
diff --cc bin/cqlsh.py
index 78fedeb,83dbeed..d007d75
--- a/bin/cqlsh.py
+++ b/bin/cqlsh.py
@@@ -480,7 -475,7 +480,7 @@@ COPY_COMMON_OPTIONS = ['DELIMITER', 'QU
                         'MAXATTEMPTS', 'REPORTFREQUENCY', 'DECIMALSEP', 'THOUSANDSSEP', 'BOOLSTYLE',
                         'NUMPROCESSES', 'CONFIGFILE', 'RATEFILE']
  COPY_FROM_OPTIONS = ['CHUNKSIZE', 'INGESTRATE', 'MAXBATCHSIZE', 'MINBATCHSIZE', 'MAXROWS',
-                      'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE', 'TTL']
 -                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE', 'PREPAREDSTATEMENTS']
++                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE', 'PREPAREDSTATEMENTS', 'TTL']
  COPY_TO_OPTIONS = ['ENCODING', 'PAGESIZE', 'PAGETIMEOUT', 'BEGINTOKEN', 'ENDTOKEN', 'MAXOUTPUTSIZE', 'MAXREQUESTS']
  
  
@@@ -1865,7 -1884,11 +1885,12 @@@ class Shell(cmd.Cmd)
            MAXINSERTERRORS=-1      - the maximum global number of insert errors, -1 means no maximum
            ERRFILE=''              - a file where to store all rows that could not be imported, by default this is
                                      import_ks_table.err where <ks> is your keyspace and <table> is your table name.
+           PREPAREDSTATEMENTS=True - whether to use prepared statements when importing, by default True. Set this to
+                                     False if you don't mind shifting data parsing to the cluster. The cluster will also
+                                     have to compile every batch statement. For large and oversized clusters
+                                     this will result in a faster import but for smaller clusters it may generate
+                                     timeouts.
 +          TTL=3600                - the time to live in seconds, by default data will not expire
  
          Available COPY TO options and defaults:
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6329d54a/pylib/cqlshlib/copyutil.py
----------------------------------------------------------------------
diff --cc pylib/cqlshlib/copyutil.py
index 95da679,6be990d..e690e82
--- a/pylib/cqlshlib/copyutil.py
+++ b/pylib/cqlshlib/copyutil.py
@@@ -187,7 -275,7 +275,8 @@@ class CopyTask(object)
          copy_options['errfile'] = safe_normpath(opts.pop('errfile', 'import_%s_%s.err' % (self.ks, self.table,)))
          copy_options['ratefile'] = safe_normpath(opts.pop('ratefile', ''))
          copy_options['maxoutputsize'] = int(opts.pop('maxoutputsize', '-1'))
+         copy_options['preparedstatements'] = bool(opts.pop('preparedstatements', 'true').lower() == 'true')
 +        copy_options['ttl'] = int(opts.pop('ttl', -1))
  
          self.check_options(copy_options)
          return CopyOptions(copy=copy_options, dialect=dialect_options, unrecognized=opts)
@@@ -1655,8 -1965,12 +1969,13 @@@ class ImportProcess(ChildProcess)
          self.max_attempts = options.copy['maxattempts']
          self.min_batch_size = options.copy['minbatchsize']
          self.max_batch_size = options.copy['maxbatchsize']
+         self.use_prepared_statements = options.copy['preparedstatements']
 +        self.ttl = options.copy['ttl']
+         self.dialect_options = options.dialect
          self._session = None
+         self.query = None
+         self.conv = None
+         self.make_statement = None
  
      @property
      def session(self):
@@@ -1700,69 -2014,88 +2019,91 @@@
              self._session.cluster.shutdown()
          ChildProcess.close(self)
  
-     def run_counter(self, table_meta):
-         """
-         Main run method for tables that contain counter columns.
-         """
-         query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
- 
-         # We prepare a query statement to find out the types of the partition key columns so we can
-         # route the update query to the correct replicas. As far as I understood this is the easiest
-         # way to find out the types of the partition columns, we will never use this prepared statement
-         where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
-         select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(self.ks), protect_name(self.table), where_clause)
-         conv = ImportConversion(self, table_meta, self.session.prepare(select_query))
- 
-         while True:
-             batch = self.inmsg.get()
-             try:
-                 for b in self.split_batches(batch, conv):
-                     self.send_counter_batch(query, conv, b)
+     def make_params(self):
+         metadata = self.session.cluster.metadata
+         table_meta = metadata.keyspaces[self.ks].tables[self.table]
+ 
+         prepared_statement = None
+         is_counter = ("counter" in [table_meta.columns[name].cql_type for name in self.valid_columns])
+         if is_counter:
+             query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
+             make_statement = self.wrap_make_statement(self.make_counter_batch_statement)
+         elif self.use_prepared_statements:
+             query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
+                                                             protect_name(self.table),
+                                                             ', '.join(protect_names(self.valid_columns),),
+                                                             ', '.join(['?' for _ in self.valid_columns]))
 -
++            if self.ttl >= 0:
++                query += 'USING TTL %s' % (self.ttl,)
+             query = self.session.prepare(query)
+             query.consistency_level = self.consistency_level
+             prepared_statement = query
+             make_statement = self.wrap_make_statement(self.make_prepared_batch_statement)
+         else:
+             query = 'INSERT INTO %s.%s (%s) VALUES (%%s)' % (protect_name(self.ks),
+                                                              protect_name(self.table),
+                                                              ', '.join(protect_names(self.valid_columns),))
++            if self.ttl >= 0:
++                query += 'USING TTL %s' % (self.ttl,)
+             make_statement = self.wrap_make_statement(self.make_non_prepared_batch_statement)
  
-             except Exception, exc:
-                 self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                 if self.debug:
-                     traceback.print_exc(exc)
+         conv = ImportConversion(self, table_meta, prepared_statement)
+         tm = TokenMap(self.ks, self.hostname, self.local_dc, self.session)
+         return query, conv, tm, make_statement
  
-     def run_normal(self, table_meta):
+     def inner_run(self, query, conv, tm, make_statement):
          """
-         Main run method for normal tables, i.e. tables that do not contain counter columns.
+         Main run method. Note that we bind self methods that are called inside loops
+         for performance reasons.
          """
-         query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
-                                                         protect_name(self.table),
-                                                         ', '.join(protect_names(self.valid_columns),),
-                                                         ', '.join(['?' for _ in self.valid_columns]))
-         if self.ttl >= 0:
-             query += 'USING TTL %s' % (self.ttl,)
+         self.query = query
+         self.conv = conv
+         self.make_statement = make_statement
  
-         query_statement = self.session.prepare(query)
-         query_statement.consistency_level = self.consistency_level
-         conv = ImportConversion(self, table_meta, query_statement)
+         convert_rows = self.convert_rows
+         split_into_batches = self.split_into_batches
+         result_callback = self.result_callback
+         err_callback = self.err_callback
+         session = self.session
  
          while True:
-             batch = self.inmsg.get()
+             chunk = self.inmsg.recv()
+             if chunk is None:
+                 break
+ 
              try:
-                 for b in self.split_batches(batch, conv):
-                     self.send_normal_batch(conv, query_statement, b)
+                 chunk['rows'] = convert_rows(conv, chunk)
+                 for replicas, batch in split_into_batches(chunk, conv, tm):
+                     statement = make_statement(query, conv, chunk, batch, replicas)
+                     future = session.execute_async(statement)
+                     future.add_callbacks(callback=result_callback, callback_args=(batch, chunk),
+                                          errback=err_callback, errback_args=(batch, chunk, replicas))
  
              except Exception, exc:
-                 self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                 if self.debug:
-                     traceback.print_exc(exc)
+                 self.report_error(exc, chunk, chunk['rows'])
  
-     def send_counter_batch(self, query_text, conv, batch):
-         if self.test_failures and self.maybe_inject_failures(batch):
-             return
+     def wrap_make_statement(self, inner_make_statement):
+         def make_statement(query, conv, chunk, batch, replicas):
+             try:
+                 return inner_make_statement(query, conv, batch, replicas)
+             except Exception, exc:
+                 print "Failed to make batch statement: {}".format(exc)
+                 self.report_error(exc, chunk, batch['rows'])
+                 return None
  
-         error_rows = []
-         batch_statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+         def make_statement_with_failures(query, conv, chunk, batch, replicas):
+             failed_batch = self.maybe_inject_failures(batch)
+             if failed_batch:
+                 return failed_batch
+             return make_statement(query, conv, chunk, batch, replicas)
  
-         for r in batch['rows']:
-             row = self.filter_row_values(r)
-             if len(row) != len(self.valid_columns):
-                 error_rows.append(row)
-                 continue
+         return make_statement_with_failures if self.test_failures else make_statement
  
+     def make_counter_batch_statement(self, query, conv, batch, replicas):
+         statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+         statement.replicas = replicas
+         statement.keyspace = self.ks
+         for row in batch['rows']:
              where_clause = []
              set_clause = []
              for i, value in enumerate(row):


[14/23] cassandra git commit: Merge commit 'c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc' into cassandra-2.2

Posted by sl...@apache.org.
Merge commit 'c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc' into cassandra-2.2


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b74ffeaf
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b74ffeaf
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b74ffeaf

Branch: refs/heads/cassandra-3.0
Commit: b74ffeafd2e08a7669013d6b4ba87980e1ad5379
Parents: 6e0395e c3d2f26
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue Mar 8 10:22:22 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Mar 8 10:23:08 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                |    1 +
 bin/cqlsh.py               |   28 +-
 pylib/cqlshlib/copyutil.py | 1160 +++++++++++++++++++++++++--------------
 pylib/cqlshlib/util.py     |   35 +-
 pylib/setup.py             |    2 +
 5 files changed, 809 insertions(+), 417 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b74ffeaf/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 89fa904,d6b085c..7b67cdc
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,30 -1,5 +1,31 @@@
 -2.1.14
 +2.2.6
 + * Fix AE in nodetool cfstats (backport CASSANDRA-10859) (CASSANDRA-11297)
 + * Unresolved hostname leads to replace being ignored (CASSANDRA-11210)
 + * Fix filtering on non-primary key columns for thrift static column families
 +   (CASSANDRA-6377)
 + * Only log yaml config once, at startup (CASSANDRA-11217)
 + * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
 + * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
 + * Range.compareTo() violates the contract of Comparable (CASSANDRA-11216)
 + * Avoid NPE when serializing ErrorMessage with null message (CASSANDRA-11167)
 + * Replacing an aggregate with a new version doesn't reset INITCOND (CASSANDRA-10840)
 + * (cqlsh) cqlsh cannot be called through symlink (CASSANDRA-11037)
 + * fix ohc and java-driver pom dependencies in build.xml (CASSANDRA-10793)
 + * Protect from keyspace dropped during repair (CASSANDRA-11065)
 + * Handle adding fields to a UDT in SELECT JSON and toJson() (CASSANDRA-11146)
 + * Better error message for cleanup (CASSANDRA-10991)
 + * cqlsh pg-style-strings broken if line ends with ';' (CASSANDRA-11123)
 + * Use cloned TokenMetadata in size estimates to avoid race against membership check
 +   (CASSANDRA-10736)
 + * Always persist upsampled index summaries (CASSANDRA-10512)
 + * (cqlsh) Fix inconsistent auto-complete (CASSANDRA-10733)
 + * Make SELECT JSON and toJson() threadsafe (CASSANDRA-11048)
 + * Fix SELECT on tuple relations for mixed ASC/DESC clustering order (CASSANDRA-7281)
 + * (cqlsh) Support utf-8/cp65001 encoding on Windows (CASSANDRA-11030)
 + * Fix paging on DISTINCT queries repeats result when first row in partition changes
 +   (CASSANDRA-10010)
 +Merged from 2.1:
+  * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
   * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
   * Don't remove FailureDetector history on removeEndpoint (CASSANDRA-10371)
   * Only notify if repair status changed (CASSANDRA-11172)


[18/23] cassandra git commit: Merge commit 'b74ffeafd2e08a7669013d6b4ba87980e1ad5379' into cassandra-3.0

Posted by sl...@apache.org.
Merge commit 'b74ffeafd2e08a7669013d6b4ba87980e1ad5379' into cassandra-3.0


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/49c616cf
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/49c616cf
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/49c616cf

Branch: refs/heads/trunk
Commit: 49c616cf0fc9c8d4649e2ec71a07fb9fd7831318
Parents: 34b07a7 b74ffea
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue Mar 8 10:23:48 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Mar 8 10:24:26 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                |    2 +-
 bin/cqlsh.py               |   33 +-
 pylib/cqlshlib/copyutil.py | 1168 +++++++++++++++++++++++++--------------
 pylib/cqlshlib/util.py     |   35 +-
 pylib/setup.py             |    2 +
 5 files changed, 821 insertions(+), 419 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/49c616cf/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 38bf2ee,7b67cdc..5b92143
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,39 -1,11 +1,39 @@@
 -2.2.6
 +3.0.5
 + * Fix filtering on non-primary key columns for queries without index (CASSANDRA-6377)
 + * Fix sstableloader fail when using materialized view (CASSANDRA-11275)
 +Merged from 2.2:
   * Fix AE in nodetool cfstats (backport CASSANDRA-10859) (CASSANDRA-11297)
   * Unresolved hostname leads to replace being ignored (CASSANDRA-11210)
 - * Fix filtering on non-primary key columns for thrift static column families
 -   (CASSANDRA-6377)
   * Only log yaml config once, at startup (CASSANDRA-11217)
 - * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
   * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
 +Merged from 2.1:
 + * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
- 
++ * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
 +
 +3.0.4
 + * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
 + * MV should only query complex columns included in the view (CASSANDRA-11069)
 + * Failed aggregate creation breaks server permanently (CASSANDRA-11064)
 + * Add sstabledump tool (CASSANDRA-7464)
 + * Introduce backpressure for hints (CASSANDRA-10972)
 + * Fix ClusteringPrefix not being able to read tombstone range boundaries (CASSANDRA-11158)
 + * Prevent logging in sandboxed state (CASSANDRA-11033)
 + * Disallow drop/alter operations of UDTs used by UDAs (CASSANDRA-10721)
 + * Add query time validation method on Index (CASSANDRA-11043)
 + * Avoid potential AssertionError in mixed version cluster (CASSANDRA-11128)
 + * Properly handle hinted handoff after topology changes (CASSANDRA-5902)
 + * AssertionError when listing sstable files on inconsistent disk state (CASSANDRA-11156)
 + * Fix wrong rack counting and invalid conditions check for TokenAllocation
 +   (CASSANDRA-11139)
 + * Avoid creating empty hint files (CASSANDRA-11090)
 + * Fix leak detection strong reference loop using weak reference (CASSANDRA-11120)
 + * Configurie BatchlogManager to stop delayed tasks on shutdown (CASSANDRA-11062)
 + * Hadoop integration is incompatible with Cassandra Driver 3.0.0 (CASSANDRA-11001)
 + * Add dropped_columns to the list of schema table so it gets handled
 +   properly (CASSANDRA-11050)
 + * Fix NPE when using forceRepairRangeAsync without DC (CASSANDRA-11239)
 +Merged from 2.2:
 + * Unresolved hostname leads to replace being ignored (CASSANDRA-11210)
   * Range.compareTo() violates the contract of Comparable (CASSANDRA-11216)
   * Avoid NPE when serializing ErrorMessage with null message (CASSANDRA-11167)
   * Replacing an aggregate with a new version doesn't reset INITCOND (CASSANDRA-10840)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/49c616cf/bin/cqlsh.py
----------------------------------------------------------------------
diff --cc bin/cqlsh.py
index 9dd9634,9082d72..83dbeed
--- a/bin/cqlsh.py
+++ b/bin/cqlsh.py
@@@ -602,24 -593,25 +602,44 @@@ def insert_driver_hooks()
  
  
  def extend_cql_deserialization():
-     # The python driver returns BLOBs as string, but we expect them as bytearrays
+     """
 -    The python driver returns BLOBs as string, but we expect them as bytearrays; therefore we change
++    The python driver returns BLOBs as string, but we expect them as bytearrays
+     the implementation of cassandra.cqltypes.BytesType.deserialize.
+ 
+     The deserializers package exists only when the driver has been compiled with cython extensions and
+     cassandra.deserializers.DesBytesType replaces cassandra.cqltypes.BytesType.deserialize.
+ 
+     DesBytesTypeByteArray is a fast deserializer that converts blobs into bytearrays but it was
+     only introduced recently (3.1.0). If it is available we use it, otherwise we remove
+     cassandra.deserializers.DesBytesType so that we fall back onto cassandra.cqltypes.BytesType.deserialize
+     just like in the case where no cython extensions are present.
+     """
+     if hasattr(cassandra, 'deserializers'):
+         if hasattr(cassandra.deserializers, 'DesBytesTypeByteArray'):
+             cassandra.deserializers.DesBytesType = cassandra.deserializers.DesBytesTypeByteArray
+         else:
+             del cassandra.deserializers.DesBytesType
+ 
      cassandra.cqltypes.BytesType.deserialize = staticmethod(lambda byts, protocol_version: bytearray(byts))
 +
 +    class DateOverFlowWarning(RuntimeWarning):
 +        pass
 +
 +    # Native datetime types blow up outside of datetime.[MIN|MAX]_YEAR. We will fall back to an int timestamp
 +    def deserialize_date_fallback_int(byts, protocol_version):
 +        timestamp_ms = int64_unpack(byts)
 +        try:
 +            return datetime_from_timestamp(timestamp_ms / 1000.0)
 +        except OverflowError:
 +            warnings.warn(DateOverFlowWarning("Some timestamps are larger than Python datetime can represent. Timestamps are displayed in milliseconds from epoch."))
 +            return timestamp_ms
 +
 +    cassandra.cqltypes.DateType.deserialize = staticmethod(deserialize_date_fallback_int)
 +
++    if hasattr(cassandra, 'deserializers'):
++        del cassandra.deserializers.DesDateType
++
 +    # Return cassandra.cqltypes.EMPTY instead of None for empty values
      cassandra.cqltypes.CassandraType.support_empty_values = True
  
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/49c616cf/pylib/cqlshlib/copyutil.py
----------------------------------------------------------------------
diff --cc pylib/cqlshlib/copyutil.py
index aeb2d0b,2755dd5..6be990d
--- a/pylib/cqlshlib/copyutil.py
+++ b/pylib/cqlshlib/copyutil.py
@@@ -1484,7 -1714,7 +1714,13 @@@ class ImportConversion(object)
  
              m = p.match(val)
              if not m:
--                raise ValueError("can't interpret %r as a date with this format: %s" % (val, self.date_time_format))
++                try:
++                    # in case of overflow COPY TO prints dates as milliseconds from the epoch, see
++                    # deserialize_date_fallback_int in cqlsh.py
++                    return int(val)
++                except ValueError:
++                    raise ValueError("can't interpret %r as a date with format %s or as int" % (val,
++                                                                                                self.date_time_format))
  
              # https://docs.python.org/2/library/time.html#time.struct_time
              tval = time.struct_time((int(m.group(1)), int(m.group(2)), int(m.group(3)),  # year, month, day


[10/23] cassandra git commit: Merge commit 'c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc' into cassandra-2.2

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b74ffeaf/bin/cqlsh.py
----------------------------------------------------------------------
diff --cc bin/cqlsh.py
index b3ec1ac,0000000..9082d72
mode 100644,000000..100644
--- a/bin/cqlsh.py
+++ b/bin/cqlsh.py
@@@ -1,2546 -1,0 +1,2566 @@@
 +#!/bin/sh
 +# -*- mode: Python -*-
 +
 +# Licensed to the Apache Software Foundation (ASF) under one
 +# or more contributor license agreements.  See the NOTICE file
 +# distributed with this work for additional information
 +# regarding copyright ownership.  The ASF licenses this file
 +# to you under the Apache License, Version 2.0 (the
 +# "License"); you may not use this file except in compliance
 +# with the License.  You may obtain a copy of the License at
 +#
 +#     http://www.apache.org/licenses/LICENSE-2.0
 +#
 +# Unless required by applicable law or agreed to in writing, software
 +# distributed under the License is distributed on an "AS IS" BASIS,
 +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 +# See the License for the specific language governing permissions and
 +# limitations under the License.
 +
 +""":"
 +# bash code here; finds a suitable python interpreter and execs this file.
 +# prefer unqualified "python" if suitable:
 +python -c 'import sys; sys.exit(not (0x020700b0 < sys.hexversion < 0x03000000))' 2>/dev/null \
 +    && exec python "$0" "$@"
 +for pyver in 2.7; do
 +    which python$pyver > /dev/null 2>&1 && exec python$pyver "$0" "$@"
 +done
 +echo "No appropriate python interpreter found." >&2
 +exit 1
 +":"""
 +
 +from __future__ import with_statement
 +
 +import cmd
 +import codecs
 +import ConfigParser
 +import csv
 +import getpass
 +import locale
 +import optparse
 +import os
 +import platform
 +import sys
 +import traceback
 +import warnings
 +import webbrowser
 +from StringIO import StringIO
 +from contextlib import contextmanager
 +from glob import glob
 +from uuid import UUID
 +
 +if sys.version_info[0] != 2 or sys.version_info[1] != 7:
 +    sys.exit("\nCQL Shell supports only Python 2.7\n")
 +
 +UTF8 = 'utf-8'
 +CP65001 = 'cp65001'  # Win utf-8 variant
 +
 +description = "CQL Shell for Apache Cassandra"
 +version = "5.0.1"
 +
 +readline = None
 +try:
 +    # check if tty first, cause readline doesn't check, and only cares
 +    # about $TERM. we don't want the funky escape code stuff to be
 +    # output if not a tty.
 +    if sys.stdin.isatty():
 +        import readline
 +except ImportError:
 +    pass
 +
 +CQL_LIB_PREFIX = 'cassandra-driver-internal-only-'
 +
 +CASSANDRA_PATH = os.path.join(os.path.dirname(os.path.realpath(__file__)), '..')
 +CASSANDRA_CQL_HTML_FALLBACK = 'https://cassandra.apache.org/doc/cql3/CQL-2.2.html'
 +
 +if os.path.exists(CASSANDRA_PATH + '/doc/cql3/CQL.html'):
 +    # default location of local CQL.html
 +    CASSANDRA_CQL_HTML = 'file://' + CASSANDRA_PATH + '/doc/cql3/CQL.html'
 +elif os.path.exists('/usr/share/doc/cassandra/CQL.html'):
 +    # fallback to package file
 +    CASSANDRA_CQL_HTML = 'file:///usr/share/doc/cassandra/CQL.html'
 +else:
 +    # fallback to online version
 +    CASSANDRA_CQL_HTML = CASSANDRA_CQL_HTML_FALLBACK
 +
 +# On Linux, the Python webbrowser module uses the 'xdg-open' executable
 +# to open a file/URL. But that only works, if the current session has been
 +# opened from _within_ a desktop environment. I.e. 'xdg-open' will fail,
 +# if the session's been opened via ssh to a remote box.
 +#
 +# Use 'python' to get some information about the detected browsers.
 +# >>> import webbrowser
 +# >>> webbrowser._tryorder
 +# >>> webbrowser._browser
 +#
 +if len(webbrowser._tryorder) == 0:
 +    CASSANDRA_CQL_HTML = CASSANDRA_CQL_HTML_FALLBACK
 +elif webbrowser._tryorder[0] == 'xdg-open' and os.environ.get('XDG_DATA_DIRS', '') == '':
 +    # only on Linux (some OS with xdg-open)
 +    webbrowser._tryorder.remove('xdg-open')
 +    webbrowser._tryorder.append('xdg-open')
 +
 +# use bundled libs for python-cql and thrift, if available. if there
 +# is a ../lib dir, use bundled libs there preferentially.
 +ZIPLIB_DIRS = [os.path.join(CASSANDRA_PATH, 'lib')]
 +myplatform = platform.system()
 +is_win = myplatform == 'Windows'
 +
 +# Workaround for supporting CP65001 encoding on python < 3.3 (https://bugs.python.org/issue13216)
 +if is_win and sys.version_info < (3, 3):
 +    codecs.register(lambda name: codecs.lookup(UTF8) if name == CP65001 else None)
 +
 +if myplatform == 'Linux':
 +    ZIPLIB_DIRS.append('/usr/share/cassandra/lib')
 +
 +if os.environ.get('CQLSH_NO_BUNDLED', ''):
 +    ZIPLIB_DIRS = ()
 +
 +
 +def find_zip(libprefix):
 +    for ziplibdir in ZIPLIB_DIRS:
 +        zips = glob(os.path.join(ziplibdir, libprefix + '*.zip'))
 +        if zips:
 +            return max(zips)   # probably the highest version, if multiple
 +
 +cql_zip = find_zip(CQL_LIB_PREFIX)
 +if cql_zip:
 +    ver = os.path.splitext(os.path.basename(cql_zip))[0][len(CQL_LIB_PREFIX):]
 +    sys.path.insert(0, os.path.join(cql_zip, 'cassandra-driver-' + ver))
 +
 +third_parties = ('futures-', 'six-')
 +
 +for lib in third_parties:
 +    lib_zip = find_zip(lib)
 +    if lib_zip:
 +        sys.path.insert(0, lib_zip)
 +
 +warnings.filterwarnings("ignore", r".*blist.*")
 +try:
 +    import cassandra
 +except ImportError, e:
 +    sys.exit("\nPython Cassandra driver not installed, or not on PYTHONPATH.\n"
 +             'You might try "pip install cassandra-driver".\n\n'
 +             'Python: %s\n'
 +             'Module load path: %r\n\n'
 +             'Error: %s\n' % (sys.executable, sys.path, e))
 +
 +from cassandra.auth import PlainTextAuthProvider
 +from cassandra.cluster import Cluster
 +from cassandra.metadata import (ColumnMetadata, KeyspaceMetadata,
 +                                TableMetadata, protect_name, protect_names)
 +from cassandra.policies import WhiteListRoundRobinPolicy
 +from cassandra.query import SimpleStatement, ordered_dict_factory, TraceUnavailable
 +
 +# cqlsh should run correctly when run out of a Cassandra source tree,
 +# out of an unpacked Cassandra tarball, and after a proper package install.
 +cqlshlibdir = os.path.join(CASSANDRA_PATH, 'pylib')
 +if os.path.isdir(cqlshlibdir):
 +    sys.path.insert(0, cqlshlibdir)
 +
 +from cqlshlib import cql3handling, cqlhandling, pylexotron, sslhandling
 +from cqlshlib.copyutil import ExportTask, ImportTask
 +from cqlshlib.displaying import (ANSI_RESET, BLUE, COLUMN_NAME_COLORS, CYAN,
 +                                 RED, FormattedValue, colorme)
 +from cqlshlib.formatting import (DEFAULT_DATE_FORMAT, DEFAULT_NANOTIME_FORMAT,
 +                                 DEFAULT_TIMESTAMP_FORMAT, DateTimeFormat,
 +                                 format_by_type, format_value_utype,
 +                                 formatter_for)
 +from cqlshlib.tracing import print_trace, print_trace_session
 +from cqlshlib.util import get_file_encoding_bomsize, trim_if_present
 +
 +DEFAULT_HOST = '127.0.0.1'
 +DEFAULT_PORT = 9042
 +DEFAULT_CQLVER = '3.3.1'
 +DEFAULT_PROTOCOL_VERSION = 4
 +DEFAULT_CONNECT_TIMEOUT_SECONDS = 5
 +DEFAULT_REQUEST_TIMEOUT_SECONDS = 10
 +
 +DEFAULT_FLOAT_PRECISION = 5
 +DEFAULT_MAX_TRACE_WAIT = 10
 +
 +if readline is not None and readline.__doc__ is not None and 'libedit' in readline.__doc__:
 +    DEFAULT_COMPLETEKEY = '\t'
 +else:
 +    DEFAULT_COMPLETEKEY = 'tab'
 +
 +cqldocs = None
 +cqlruleset = None
 +
 +epilog = """Connects to %(DEFAULT_HOST)s:%(DEFAULT_PORT)d by default. These
 +defaults can be changed by setting $CQLSH_HOST and/or $CQLSH_PORT. When a
 +host (and optional port number) are given on the command line, they take
 +precedence over any defaults.""" % globals()
 +
 +parser = optparse.OptionParser(description=description, epilog=epilog,
 +                               usage="Usage: %prog [options] [host [port]]",
 +                               version='cqlsh ' + version)
 +parser.add_option("-C", "--color", action='store_true', dest='color',
 +                  help='Always use color output')
 +parser.add_option("--no-color", action='store_false', dest='color',
 +                  help='Never use color output')
 +parser.add_option("--browser", dest='browser', help="""The browser to use to display CQL help, where BROWSER can be:
 +                                                    - one of the supported browsers in https://docs.python.org/2/library/webbrowser.html.
 +                                                    - browser path followed by %s, example: /usr/bin/google-chrome-stable %s""")
 +parser.add_option('--ssl', action='store_true', help='Use SSL', default=False)
 +parser.add_option("-u", "--username", help="Authenticate as user.")
 +parser.add_option("-p", "--password", help="Authenticate using password.")
 +parser.add_option('-k', '--keyspace', help='Authenticate to the given keyspace.')
 +parser.add_option("-f", "--file", help="Execute commands from FILE, then exit")
 +parser.add_option('--debug', action='store_true',
 +                  help='Show additional debugging information')
 +parser.add_option("--encoding", help="Specify a non-default encoding for output." +
 +                  " (Default: %s)" % (UTF8,))
 +parser.add_option("--cqlshrc", help="Specify an alternative cqlshrc file location.")
 +parser.add_option('--cqlversion', default=DEFAULT_CQLVER,
 +                  help='Specify a particular CQL version (default: %default).'
 +                       ' Examples: "3.0.3", "3.1.0"')
 +parser.add_option("-e", "--execute", help='Execute the statement and quit.')
 +parser.add_option("--connect-timeout", default=DEFAULT_CONNECT_TIMEOUT_SECONDS, dest='connect_timeout',
 +                  help='Specify the connection timeout in seconds (default: %default seconds).')
 +parser.add_option("--request-timeout", default=DEFAULT_REQUEST_TIMEOUT_SECONDS, dest='request_timeout',
 +                  help='Specify the default request timeout in seconds (default: %default seconds).')
 +parser.add_option("-t", "--tty", action='store_true', dest='tty',
 +                  help='Force tty mode (command prompt).')
 +
 +optvalues = optparse.Values()
 +(options, arguments) = parser.parse_args(sys.argv[1:], values=optvalues)
 +
 +# BEGIN history/config definition
 +HISTORY_DIR = os.path.expanduser(os.path.join('~', '.cassandra'))
 +
 +if hasattr(options, 'cqlshrc'):
 +    CONFIG_FILE = options.cqlshrc
 +    if not os.path.exists(CONFIG_FILE):
 +        print '\nWarning: Specified cqlshrc location `%s` does not exist.  Using `%s` instead.\n' % (CONFIG_FILE, HISTORY_DIR)
 +        CONFIG_FILE = os.path.join(HISTORY_DIR, 'cqlshrc')
 +else:
 +    CONFIG_FILE = os.path.join(HISTORY_DIR, 'cqlshrc')
 +
 +HISTORY = os.path.join(HISTORY_DIR, 'cqlsh_history')
 +if not os.path.exists(HISTORY_DIR):
 +    try:
 +        os.mkdir(HISTORY_DIR)
 +    except OSError:
 +        print '\nWarning: Cannot create directory at `%s`. Command history will not be saved.\n' % HISTORY_DIR
 +
 +OLD_CONFIG_FILE = os.path.expanduser(os.path.join('~', '.cqlshrc'))
 +if os.path.exists(OLD_CONFIG_FILE):
 +    if os.path.exists(CONFIG_FILE):
 +        print '\nWarning: cqlshrc config files were found at both the old location (%s) and \
 +                the new location (%s), the old config file will not be migrated to the new \
 +                location, and the new location will be used for now.  You should manually \
 +                consolidate the config files at the new location and remove the old file.' \
 +                % (OLD_CONFIG_FILE, CONFIG_FILE)
 +    else:
 +        os.rename(OLD_CONFIG_FILE, CONFIG_FILE)
 +OLD_HISTORY = os.path.expanduser(os.path.join('~', '.cqlsh_history'))
 +if os.path.exists(OLD_HISTORY):
 +    os.rename(OLD_HISTORY, HISTORY)
 +# END history/config definition
 +
 +CQL_ERRORS = (
 +    cassandra.AlreadyExists, cassandra.AuthenticationFailed, cassandra.InvalidRequest,
 +    cassandra.Timeout, cassandra.Unauthorized, cassandra.OperationTimedOut,
 +    cassandra.cluster.NoHostAvailable,
 +    cassandra.connection.ConnectionBusy, cassandra.connection.ProtocolError, cassandra.connection.ConnectionException,
 +    cassandra.protocol.ErrorMessage, cassandra.protocol.InternalError, cassandra.query.TraceUnavailable
 +)
 +
 +debug_completion = bool(os.environ.get('CQLSH_DEBUG_COMPLETION', '') == 'YES')
 +
 +# we want the cql parser to understand our cqlsh-specific commands too
 +my_commands_ending_with_newline = (
 +    'help',
 +    '?',
 +    'consistency',
 +    'serial',
 +    'describe',
 +    'desc',
 +    'show',
 +    'source',
 +    'capture',
 +    'login',
 +    'debug',
 +    'tracing',
 +    'expand',
 +    'paging',
 +    'exit',
 +    'quit',
 +    'clear',
 +    'cls'
 +)
 +
 +
 +cqlsh_syntax_completers = []
 +
 +
 +def cqlsh_syntax_completer(rulename, termname):
 +    def registrator(f):
 +        cqlsh_syntax_completers.append((rulename, termname, f))
 +        return f
 +    return registrator
 +
 +
 +cqlsh_extra_syntax_rules = r'''
 +<cqlshCommand> ::= <CQL_Statement>
 +                 | <specialCommand> ( ";" | "\n" )
 +                 ;
 +
 +<specialCommand> ::= <describeCommand>
 +                   | <consistencyCommand>
 +                   | <serialConsistencyCommand>
 +                   | <showCommand>
 +                   | <sourceCommand>
 +                   | <captureCommand>
 +                   | <copyCommand>
 +                   | <loginCommand>
 +                   | <debugCommand>
 +                   | <helpCommand>
 +                   | <tracingCommand>
 +                   | <expandCommand>
 +                   | <exitCommand>
 +                   | <pagingCommand>
 +                   | <clearCommand>
 +                   ;
 +
 +<describeCommand> ::= ( "DESCRIBE" | "DESC" )
 +                                  ( "FUNCTIONS"
 +                                  | "FUNCTION" udf=<anyFunctionName>
 +                                  | "AGGREGATES"
 +                                  | "AGGREGATE" uda=<userAggregateName>
 +                                  | "KEYSPACES"
 +                                  | "KEYSPACE" ksname=<keyspaceName>?
 +                                  | ( "COLUMNFAMILY" | "TABLE" ) cf=<columnFamilyName>
 +                                  | "INDEX" idx=<indexName>
 +                                  | ( "COLUMNFAMILIES" | "TABLES" )
 +                                  | "FULL"? "SCHEMA"
 +                                  | "CLUSTER"
 +                                  | "TYPES"
 +                                  | "TYPE" ut=<userTypeName>
 +                                  | (ksname=<keyspaceName> | cf=<columnFamilyName> | idx=<indexName>))
 +                    ;
 +
 +<consistencyCommand> ::= "CONSISTENCY" ( level=<consistencyLevel> )?
 +                       ;
 +
 +<consistencyLevel> ::= "ANY"
 +                     | "ONE"
 +                     | "TWO"
 +                     | "THREE"
 +                     | "QUORUM"
 +                     | "ALL"
 +                     | "LOCAL_QUORUM"
 +                     | "EACH_QUORUM"
 +                     | "SERIAL"
 +                     | "LOCAL_SERIAL"
 +                     | "LOCAL_ONE"
 +                     ;
 +
 +<serialConsistencyCommand> ::= "SERIAL" "CONSISTENCY" ( level=<serialConsistencyLevel> )?
 +                             ;
 +
 +<serialConsistencyLevel> ::= "SERIAL"
 +                           | "LOCAL_SERIAL"
 +                           ;
 +
 +<showCommand> ::= "SHOW" what=( "VERSION" | "HOST" | "SESSION" sessionid=<uuid> )
 +                ;
 +
 +<sourceCommand> ::= "SOURCE" fname=<stringLiteral>
 +                  ;
 +
 +<captureCommand> ::= "CAPTURE" ( fname=( <stringLiteral> | "OFF" ) )?
 +                   ;
 +
 +<copyCommand> ::= "COPY" cf=<columnFamilyName>
 +                         ( "(" [colnames]=<colname> ( "," [colnames]=<colname> )* ")" )?
 +                         ( dir="FROM" ( fname=<stringLiteral> | "STDIN" )
 +                         | dir="TO"   ( fname=<stringLiteral> | "STDOUT" ) )
 +                         ( "WITH" <copyOption> ( "AND" <copyOption> )* )?
 +                ;
 +
 +<copyOption> ::= [optnames]=(<identifier>|<reserved_identifier>) "=" [optvals]=<copyOptionVal>
 +               ;
 +
 +<copyOptionVal> ::= <identifier>
 +                  | <reserved_identifier>
 +                  | <term>
 +                  ;
 +
 +# avoiding just "DEBUG" so that this rule doesn't get treated as a terminal
 +<debugCommand> ::= "DEBUG" "THINGS"?
 +                 ;
 +
 +<helpCommand> ::= ( "HELP" | "?" ) [topic]=( /[a-z_]*/ )*
 +                ;
 +
 +<tracingCommand> ::= "TRACING" ( switch=( "ON" | "OFF" ) )?
 +                   ;
 +
 +<expandCommand> ::= "EXPAND" ( switch=( "ON" | "OFF" ) )?
 +                   ;
 +
 +<pagingCommand> ::= "PAGING" ( switch=( "ON" | "OFF" | /[0-9]+/) )?
 +                  ;
 +
 +<loginCommand> ::= "LOGIN" username=<username> (password=<stringLiteral>)?
 +                 ;
 +
 +<exitCommand> ::= "exit" | "quit"
 +                ;
 +
 +<clearCommand> ::= "CLEAR" | "CLS"
 +                 ;
 +
 +<qmark> ::= "?" ;
 +'''
 +
 +
 +@cqlsh_syntax_completer('helpCommand', 'topic')
 +def complete_help(ctxt, cqlsh):
 +    return sorted([t.upper() for t in cqldocs.get_help_topics() + cqlsh.get_help_topics()])
 +
 +
 +def complete_source_quoted_filename(ctxt, cqlsh):
 +    partial_path = ctxt.get_binding('partial', '')
 +    head, tail = os.path.split(partial_path)
 +    exhead = os.path.expanduser(head)
 +    try:
 +        contents = os.listdir(exhead or '.')
 +    except OSError:
 +        return ()
 +    matches = filter(lambda f: f.startswith(tail), contents)
 +    annotated = []
 +    for f in matches:
 +        match = os.path.join(head, f)
 +        if os.path.isdir(os.path.join(exhead, f)):
 +            match += '/'
 +        annotated.append(match)
 +    return annotated
 +
 +
 +cqlsh_syntax_completer('sourceCommand', 'fname')(complete_source_quoted_filename)
 +cqlsh_syntax_completer('captureCommand', 'fname')(complete_source_quoted_filename)
 +
 +
 +@cqlsh_syntax_completer('copyCommand', 'fname')
 +def copy_fname_completer(ctxt, cqlsh):
 +    lasttype = ctxt.get_binding('*LASTTYPE*')
 +    if lasttype == 'unclosedString':
 +        return complete_source_quoted_filename(ctxt, cqlsh)
 +    partial_path = ctxt.get_binding('partial')
 +    if partial_path == '':
 +        return ["'"]
 +    return ()
 +
 +
 +@cqlsh_syntax_completer('copyCommand', 'colnames')
 +def complete_copy_column_names(ctxt, cqlsh):
 +    existcols = map(cqlsh.cql_unprotect_name, ctxt.get_binding('colnames', ()))
 +    ks = cqlsh.cql_unprotect_name(ctxt.get_binding('ksname', None))
 +    cf = cqlsh.cql_unprotect_name(ctxt.get_binding('cfname'))
 +    colnames = cqlsh.get_column_names(ks, cf)
 +    if len(existcols) == 0:
 +        return [colnames[0]]
 +    return set(colnames[1:]) - set(existcols)
 +
 +
 +COPY_COMMON_OPTIONS = ['DELIMITER', 'QUOTE', 'ESCAPE', 'HEADER', 'NULL', 'DATETIMEFORMAT',
 +                       'MAXATTEMPTS', 'REPORTFREQUENCY', 'DECIMALSEP', 'THOUSANDSSEP', 'BOOLSTYLE',
 +                       'NUMPROCESSES', 'CONFIGFILE', 'RATEFILE']
 +COPY_FROM_OPTIONS = ['CHUNKSIZE', 'INGESTRATE', 'MAXBATCHSIZE', 'MINBATCHSIZE', 'MAXROWS',
-                      'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE']
++                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE', 'PREPAREDSTATEMENTS']
 +COPY_TO_OPTIONS = ['ENCODING', 'PAGESIZE', 'PAGETIMEOUT', 'BEGINTOKEN', 'ENDTOKEN', 'MAXOUTPUTSIZE', 'MAXREQUESTS']
 +
 +
 +@cqlsh_syntax_completer('copyOption', 'optnames')
 +def complete_copy_options(ctxt, cqlsh):
 +    optnames = map(str.upper, ctxt.get_binding('optnames', ()))
 +    direction = ctxt.get_binding('dir').upper()
 +    if direction == 'FROM':
 +        opts = set(COPY_COMMON_OPTIONS + COPY_FROM_OPTIONS) - set(optnames)
 +    elif direction == 'TO':
 +        opts = set(COPY_COMMON_OPTIONS + COPY_TO_OPTIONS) - set(optnames)
 +    return opts
 +
 +
 +@cqlsh_syntax_completer('copyOption', 'optvals')
 +def complete_copy_opt_values(ctxt, cqlsh):
 +    optnames = ctxt.get_binding('optnames', ())
 +    lastopt = optnames[-1].lower()
 +    if lastopt == 'header':
 +        return ['true', 'false']
 +    return [cqlhandling.Hint('<single_character_string>')]
 +
 +
 +class NoKeyspaceError(Exception):
 +    pass
 +
 +
 +class KeyspaceNotFound(Exception):
 +    pass
 +
 +
 +class ColumnFamilyNotFound(Exception):
 +    pass
 +
 +
 +class IndexNotFound(Exception):
 +    pass
 +
 +
 +class ObjectNotFound(Exception):
 +    pass
 +
 +
 +class VersionNotSupported(Exception):
 +    pass
 +
 +
 +class UserTypeNotFound(Exception):
 +    pass
 +
 +
 +class FunctionNotFound(Exception):
 +    pass
 +
 +
 +class AggregateNotFound(Exception):
 +    pass
 +
 +
 +class DecodeError(Exception):
 +    verb = 'decode'
 +
 +    def __init__(self, thebytes, err, colname=None):
 +        self.thebytes = thebytes
 +        self.err = err
 +        self.colname = colname
 +
 +    def __str__(self):
 +        return str(self.thebytes)
 +
 +    def message(self):
 +        what = 'value %r' % (self.thebytes,)
 +        if self.colname is not None:
 +            what = 'value %r (for column %r)' % (self.thebytes, self.colname)
 +        return 'Failed to %s %s : %s' \
 +               % (self.verb, what, self.err)
 +
 +    def __repr__(self):
 +        return '<%s %s>' % (self.__class__.__name__, self.message())
 +
 +
 +class FormatError(DecodeError):
 +    verb = 'format'
 +
 +
 +def full_cql_version(ver):
 +    while ver.count('.') < 2:
 +        ver += '.0'
 +    ver_parts = ver.split('-', 1) + ['']
 +    vertuple = tuple(map(int, ver_parts[0].split('.')) + [ver_parts[1]])
 +    return ver, vertuple
 +
 +
 +def format_value(val, output_encoding, addcolor=False, date_time_format=None,
 +                 float_precision=None, colormap=None, nullval=None):
 +    if isinstance(val, DecodeError):
 +        if addcolor:
 +            return colorme(repr(val.thebytes), colormap, 'error')
 +        else:
 +            return FormattedValue(repr(val.thebytes))
 +    return format_by_type(type(val), val, output_encoding, colormap=colormap,
 +                          addcolor=addcolor, nullval=nullval, date_time_format=date_time_format,
 +                          float_precision=float_precision)
 +
 +
 +def show_warning_without_quoting_line(message, category, filename, lineno, file=None, line=None):
 +    if file is None:
 +        file = sys.stderr
 +    try:
 +        file.write(warnings.formatwarning(message, category, filename, lineno, line=''))
 +    except IOError:
 +        pass
 +warnings.showwarning = show_warning_without_quoting_line
 +warnings.filterwarnings('always', category=cql3handling.UnexpectedTableStructure)
 +
 +
 +def insert_driver_hooks():
 +    extend_cql_deserialization()
 +    auto_format_udts()
 +
 +
 +def extend_cql_deserialization():
 +    """
-     The python driver returns BLOBs as string, but we expect them as bytearrays
++    The python driver returns BLOBs as string, but we expect them as bytearrays; therefore we change
++    the implementation of cassandra.cqltypes.BytesType.deserialize.
++
++    The deserializers package exists only when the driver has been compiled with cython extensions and
++    cassandra.deserializers.DesBytesType replaces cassandra.cqltypes.BytesType.deserialize.
++
++    DesBytesTypeByteArray is a fast deserializer that converts blobs into bytearrays but it was
++    only introduced recently (3.1.0). If it is available we use it, otherwise we remove
++    cassandra.deserializers.DesBytesType so that we fall back onto cassandra.cqltypes.BytesType.deserialize
++    just like in the case where no cython extensions are present.
 +    """
++    if hasattr(cassandra, 'deserializers'):
++        if hasattr(cassandra.deserializers, 'DesBytesTypeByteArray'):
++            cassandra.deserializers.DesBytesType = cassandra.deserializers.DesBytesTypeByteArray
++        else:
++            del cassandra.deserializers.DesBytesType
++
 +    cassandra.cqltypes.BytesType.deserialize = staticmethod(lambda byts, protocol_version: bytearray(byts))
 +    cassandra.cqltypes.CassandraType.support_empty_values = True
 +
 +
 +def auto_format_udts():
 +    # when we see a new user defined type, set up the shell formatting for it
 +    udt_apply_params = cassandra.cqltypes.UserType.apply_parameters
 +
 +    def new_apply_params(cls, *args, **kwargs):
 +        udt_class = udt_apply_params(*args, **kwargs)
 +        formatter_for(udt_class.typename)(format_value_utype)
 +        return udt_class
 +
 +    cassandra.cqltypes.UserType.udt_apply_parameters = classmethod(new_apply_params)
 +
 +    make_udt_class = cassandra.cqltypes.UserType.make_udt_class
 +
 +    def new_make_udt_class(cls, *args, **kwargs):
 +        udt_class = make_udt_class(*args, **kwargs)
 +        formatter_for(udt_class.tuple_type.__name__)(format_value_utype)
 +        return udt_class
 +
 +    cassandra.cqltypes.UserType.make_udt_class = classmethod(new_make_udt_class)
 +
 +
 +class FrozenType(cassandra.cqltypes._ParameterizedType):
 +    """
 +    Needed until the bundled python driver adds FrozenType.
 +    """
 +    typename = "frozen"
 +    num_subtypes = 1
 +
 +    @classmethod
 +    def deserialize_safe(cls, byts, protocol_version):
 +        subtype, = cls.subtypes
 +        return subtype.from_binary(byts)
 +
 +    @classmethod
 +    def serialize_safe(cls, val, protocol_version):
 +        subtype, = cls.subtypes
 +        return subtype.to_binary(val, protocol_version)
 +
 +
 +class Shell(cmd.Cmd):
 +    custom_prompt = os.getenv('CQLSH_PROMPT', '')
 +    if custom_prompt is not '':
 +        custom_prompt += "\n"
 +    default_prompt = custom_prompt + "cqlsh> "
 +    continue_prompt = "   ... "
 +    keyspace_prompt = custom_prompt + "cqlsh:%s> "
 +    keyspace_continue_prompt = "%s    ... "
 +    show_line_nums = False
 +    debug = False
 +    stop = False
 +    last_hist = None
 +    shunted_query_out = None
 +    use_paging = True
 +
 +    default_page_size = 100
 +
 +    def __init__(self, hostname, port, color=False,
 +                 username=None, password=None, encoding=None, stdin=None, tty=True,
 +                 completekey=DEFAULT_COMPLETEKEY, browser=None, use_conn=None,
 +                 cqlver=DEFAULT_CQLVER, keyspace=None,
 +                 tracing_enabled=False, expand_enabled=False,
 +                 display_nanotime_format=DEFAULT_NANOTIME_FORMAT,
 +                 display_timestamp_format=DEFAULT_TIMESTAMP_FORMAT,
 +                 display_date_format=DEFAULT_DATE_FORMAT,
 +                 display_float_precision=DEFAULT_FLOAT_PRECISION,
 +                 display_timezone=None,
 +                 max_trace_wait=DEFAULT_MAX_TRACE_WAIT,
 +                 ssl=False,
 +                 single_statement=None,
 +                 request_timeout=DEFAULT_REQUEST_TIMEOUT_SECONDS,
 +                 protocol_version=DEFAULT_PROTOCOL_VERSION,
 +                 connect_timeout=DEFAULT_CONNECT_TIMEOUT_SECONDS):
 +        cmd.Cmd.__init__(self, completekey=completekey)
 +        self.hostname = hostname
 +        self.port = port
 +        self.auth_provider = None
 +        if username:
 +            if not password:
 +                password = getpass.getpass()
 +            self.auth_provider = PlainTextAuthProvider(username=username, password=password)
 +        self.username = username
 +        self.keyspace = keyspace
 +        self.ssl = ssl
 +        self.tracing_enabled = tracing_enabled
 +        self.page_size = self.default_page_size
 +        self.expand_enabled = expand_enabled
 +        if use_conn:
 +            self.conn = use_conn
 +        else:
 +            self.conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=cqlver,
 +                                protocol_version=protocol_version,
 +                                auth_provider=self.auth_provider,
 +                                ssl_options=sslhandling.ssl_settings(hostname, CONFIG_FILE) if ssl else None,
 +                                load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]),
 +                                control_connection_timeout=connect_timeout,
 +                                connect_timeout=connect_timeout)
 +        self.owns_connection = not use_conn
 +        self.set_expanded_cql_version(cqlver)
 +
 +        if keyspace:
 +            self.session = self.conn.connect(keyspace)
 +        else:
 +            self.session = self.conn.connect()
 +
 +        if browser == "":
 +            browser = None
 +        self.browser = browser
 +        self.color = color
 +
 +        self.display_nanotime_format = display_nanotime_format
 +        self.display_timestamp_format = display_timestamp_format
 +        self.display_date_format = display_date_format
 +
 +        self.display_float_precision = display_float_precision
 +
 +        self.display_timezone = display_timezone
 +
 +        # If there is no schema metadata present (due to a schema mismatch), force schema refresh
 +        if not self.conn.metadata.keyspaces:
 +            self.refresh_schema_metadata_best_effort()
 +
 +        self.session.default_timeout = request_timeout
 +        self.session.row_factory = ordered_dict_factory
 +        self.session.default_consistency_level = cassandra.ConsistencyLevel.ONE
 +        self.get_connection_versions()
 +
 +        self.current_keyspace = keyspace
 +
 +        self.display_timestamp_format = display_timestamp_format
 +        self.display_nanotime_format = display_nanotime_format
 +        self.display_date_format = display_date_format
 +
 +        self.max_trace_wait = max_trace_wait
 +        self.session.max_trace_wait = max_trace_wait
 +
 +        self.tty = tty
 +        self.encoding = encoding
 +        self.check_windows_encoding()
 +
 +        self.output_codec = codecs.lookup(encoding)
 +
 +        self.statement = StringIO()
 +        self.lineno = 1
 +        self.in_comment = False
 +
 +        self.prompt = ''
 +        if stdin is None:
 +            stdin = sys.stdin
 +
 +        if tty:
 +            self.reset_prompt()
 +            self.report_connection()
 +            print 'Use HELP for help.'
 +        else:
 +            self.show_line_nums = True
 +        self.stdin = stdin
 +        self.query_out = sys.stdout
 +        self.consistency_level = cassandra.ConsistencyLevel.ONE
 +        self.serial_consistency_level = cassandra.ConsistencyLevel.SERIAL
 +
 +        self.empty_lines = 0
 +        self.statement_error = False
 +        self.single_statement = single_statement
 +
 +    @property
 +    def is_using_utf8(self):
 +        # utf8 encodings from https://docs.python.org/{2,3}/library/codecs.html
 +        return self.encoding.replace('-', '_').lower() in ['utf', 'utf_8', 'u8', 'utf8', CP65001]
 +
 +    def check_windows_encoding(self):
 +        if is_win and os.name == 'nt' and self.tty and \
 +           self.is_using_utf8 and sys.stdout.encoding != CP65001:
 +            self.printerr("\nWARNING: console codepage must be set to cp65001 "
 +                          "to support {} encoding on Windows platforms.\n"
 +                          "If you experience encoding problems, change your console"
 +                          " codepage with 'chcp 65001' before starting cqlsh.\n".format(self.encoding))
 +
 +    def refresh_schema_metadata_best_effort(self):
 +        try:
 +            self.conn.refresh_schema_metadata(5)  # will throw exception if there is a schema mismatch
 +        except Exception:
 +            self.printerr("Warning: schema version mismatch detected, which might be caused by DOWN nodes; if "
 +                          "this is not the case, check the schema versions of your nodes in system.local and "
 +                          "system.peers.")
 +            self.conn.refresh_schema_metadata(0)
 +
 +    def set_expanded_cql_version(self, ver):
 +        ver, vertuple = full_cql_version(ver)
 +        self.cql_version = ver
 +        self.cql_ver_tuple = vertuple
 +
 +    def cqlver_atleast(self, major, minor=0, patch=0):
 +        return self.cql_ver_tuple[:3] >= (major, minor, patch)
 +
 +    def myformat_value(self, val, **kwargs):
 +        if isinstance(val, DecodeError):
 +            self.decoding_errors.append(val)
 +        try:
 +            dtformats = DateTimeFormat(timestamp_format=self.display_timestamp_format,
 +                                       date_format=self.display_date_format, nanotime_format=self.display_nanotime_format,
 +                                       timezone=self.display_timezone)
 +            return format_value(val, self.output_codec.name,
 +                                addcolor=self.color, date_time_format=dtformats,
 +                                float_precision=self.display_float_precision, **kwargs)
 +        except Exception, e:
 +            err = FormatError(val, e)
 +            self.decoding_errors.append(err)
 +            return format_value(err, self.output_codec.name, addcolor=self.color)
 +
 +    def myformat_colname(self, name, table_meta=None):
 +        column_colors = COLUMN_NAME_COLORS.copy()
 +        # check column role and color appropriately
 +        if table_meta:
 +            if name in [col.name for col in table_meta.partition_key]:
 +                column_colors.default_factory = lambda: RED
 +            elif name in [col.name for col in table_meta.clustering_key]:
 +                column_colors.default_factory = lambda: CYAN
 +        return self.myformat_value(name, colormap=column_colors)
 +
 +    def report_connection(self):
 +        self.show_host()
 +        self.show_version()
 +
 +    def show_host(self):
 +        print "Connected to %s at %s:%d." % \
 +            (self.applycolor(self.get_cluster_name(), BLUE),
 +              self.hostname,
 +              self.port)
 +
 +    def show_version(self):
 +        vers = self.connection_versions.copy()
 +        vers['shver'] = version
 +        # system.Versions['cql'] apparently does not reflect changes with
 +        # set_cql_version.
 +        vers['cql'] = self.cql_version
 +        print "[cqlsh %(shver)s | Cassandra %(build)s | CQL spec %(cql)s | Native protocol v%(protocol)s]" % vers
 +
 +    def show_session(self, sessionid, partial_session=False):
 +        print_trace_session(self, self.session, sessionid, partial_session)
 +
 +    def get_connection_versions(self):
 +        result, = self.session.execute("select * from system.local where key = 'local'")
 +        vers = {
 +            'build': result['release_version'],
 +            'protocol': result['native_protocol_version'],
 +            'cql': result['cql_version'],
 +        }
 +        self.connection_versions = vers
 +
 +    def get_keyspace_names(self):
 +        return map(str, self.conn.metadata.keyspaces.keys())
 +
 +    def get_columnfamily_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return map(str, self.get_keyspace_meta(ksname).tables.keys())
 +
 +    def get_index_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return map(str, self.get_keyspace_meta(ksname).indexes.keys())
 +
 +    def get_column_names(self, ksname, cfname):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        layout = self.get_table_meta(ksname, cfname)
 +        return [str(col) for col in layout.columns]
 +
 +    def get_usertype_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return self.get_keyspace_meta(ksname).user_types.keys()
 +
 +    def get_usertype_layout(self, ksname, typename):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        ks_meta = self.get_keyspace_meta(ksname)
 +
 +        try:
 +            user_type = ks_meta.user_types[typename]
 +        except KeyError:
 +            raise UserTypeNotFound("User type %r not found" % typename)
 +
 +        return [(field_name, field_type.cql_parameterized_type())
 +                for field_name, field_type in zip(user_type.field_names, user_type.field_types)]
 +
 +    def get_userfunction_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return map(lambda f: f.name, self.get_keyspace_meta(ksname).functions.values())
 +
 +    def get_useraggregate_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return map(lambda f: f.name, self.get_keyspace_meta(ksname).aggregates.values())
 +
 +    def get_cluster_name(self):
 +        return self.conn.metadata.cluster_name
 +
 +    def get_partitioner(self):
 +        return self.conn.metadata.partitioner
 +
 +    def get_keyspace_meta(self, ksname):
 +        if ksname not in self.conn.metadata.keyspaces:
 +            raise KeyspaceNotFound('Keyspace %r not found.' % ksname)
 +        return self.conn.metadata.keyspaces[ksname]
 +
 +    def get_keyspaces(self):
 +        return self.conn.metadata.keyspaces.values()
 +
 +    def get_ring(self, ks):
 +        self.conn.metadata.token_map.rebuild_keyspace(ks, build_if_absent=True)
 +        return self.conn.metadata.token_map.tokens_to_hosts_by_ks[ks]
 +
 +    def get_table_meta(self, ksname, tablename):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        ksmeta = self.get_keyspace_meta(ksname)
 +
 +        if tablename not in ksmeta.tables:
 +            if ksname == 'system_auth' and tablename in ['roles', 'role_permissions']:
 +                self.get_fake_auth_table_meta(ksname, tablename)
 +            else:
 +                raise ColumnFamilyNotFound("Column family %r not found" % tablename)
 +        else:
 +            return ksmeta.tables[tablename]
 +
 +    def get_fake_auth_table_meta(self, ksname, tablename):
 +        # may be using external auth implementation so internal tables
 +        # aren't actually defined in schema. In this case, we'll fake
 +        # them up
 +        if tablename == 'roles':
 +            ks_meta = KeyspaceMetadata(ksname, True, None, None)
 +            table_meta = TableMetadata(ks_meta, 'roles')
 +            table_meta.columns['role'] = ColumnMetadata(table_meta, 'role', cassandra.cqltypes.UTF8Type)
 +            table_meta.columns['is_superuser'] = ColumnMetadata(table_meta, 'is_superuser', cassandra.cqltypes.BooleanType)
 +            table_meta.columns['can_login'] = ColumnMetadata(table_meta, 'can_login', cassandra.cqltypes.BooleanType)
 +        elif tablename == 'role_permissions':
 +            ks_meta = KeyspaceMetadata(ksname, True, None, None)
 +            table_meta = TableMetadata(ks_meta, 'role_permissions')
 +            table_meta.columns['role'] = ColumnMetadata(table_meta, 'role', cassandra.cqltypes.UTF8Type)
 +            table_meta.columns['resource'] = ColumnMetadata(table_meta, 'resource', cassandra.cqltypes.UTF8Type)
 +            table_meta.columns['permission'] = ColumnMetadata(table_meta, 'permission', cassandra.cqltypes.UTF8Type)
 +        else:
 +            raise ColumnFamilyNotFound("Column family %r not found" % tablename)
 +
 +    def get_index_meta(self, ksname, idxname):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        ksmeta = self.get_keyspace_meta(ksname)
 +
 +        if idxname not in ksmeta.indexes:
 +            raise IndexNotFound("Index %r not found" % idxname)
 +
 +        return ksmeta.indexes[idxname]
 +
 +    def get_object_meta(self, ks, name):
 +        if name is None:
 +            if ks and ks in self.conn.metadata.keyspaces:
 +                return self.conn.metadata.keyspaces[ks]
 +            elif self.current_keyspace is None:
 +                raise ObjectNotFound("%r not found in keyspaces" % (ks))
 +            else:
 +                name = ks
 +                ks = self.current_keyspace
 +
 +        if ks is None:
 +            ks = self.current_keyspace
 +
 +        ksmeta = self.get_keyspace_meta(ks)
 +
 +        if name in ksmeta.tables:
 +            return ksmeta.tables[name]
 +        elif name in ksmeta.indexes:
 +            return ksmeta.indexes[name]
 +
 +        raise ObjectNotFound("%r not found in keyspace %r" % (name, ks))
 +
 +    def get_usertypes_meta(self):
 +        data = self.session.execute("select * from system.schema_usertypes")
 +        if not data:
 +            return cql3handling.UserTypesMeta({})
 +
 +        return cql3handling.UserTypesMeta.from_layout(data)
 +
 +    def get_trigger_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return [trigger.name
 +                for table in self.get_keyspace_meta(ksname).tables.values()
 +                for trigger in table.triggers.values()]
 +
 +    def reset_statement(self):
 +        self.reset_prompt()
 +        self.statement.truncate(0)
 +        self.empty_lines = 0
 +
 +    def reset_prompt(self):
 +        if self.current_keyspace is None:
 +            self.set_prompt(self.default_prompt, True)
 +        else:
 +            self.set_prompt(self.keyspace_prompt % self.current_keyspace, True)
 +
 +    def set_continue_prompt(self):
 +        if self.empty_lines >= 3:
 +            self.set_prompt("Statements are terminated with a ';'.  You can press CTRL-C to cancel an incomplete statement.")
 +            self.empty_lines = 0
 +            return
 +        if self.current_keyspace is None:
 +            self.set_prompt(self.continue_prompt)
 +        else:
 +            spaces = ' ' * len(str(self.current_keyspace))
 +            self.set_prompt(self.keyspace_continue_prompt % spaces)
 +        self.empty_lines = self.empty_lines + 1 if not self.lastcmd else 0
 +
 +    @contextmanager
 +    def prepare_loop(self):
 +        readline = None
 +        if self.tty and self.completekey:
 +            try:
 +                import readline
 +            except ImportError:
 +                if is_win:
 +                    print "WARNING: pyreadline dependency missing.  Install to enable tab completion."
 +                pass
 +            else:
 +                old_completer = readline.get_completer()
 +                readline.set_completer(self.complete)
 +                if readline.__doc__ is not None and 'libedit' in readline.__doc__:
 +                    readline.parse_and_bind("bind -e")
 +                    readline.parse_and_bind("bind '" + self.completekey + "' rl_complete")
 +                    readline.parse_and_bind("bind ^R em-inc-search-prev")
 +                else:
 +                    readline.parse_and_bind(self.completekey + ": complete")
 +        try:
 +            yield
 +        finally:
 +            if readline is not None:
 +                readline.set_completer(old_completer)
 +
 +    def get_input_line(self, prompt=''):
 +        if self.tty:
 +            try:
 +                self.lastcmd = raw_input(prompt).decode(self.encoding)
 +            except UnicodeDecodeError:
 +                self.lastcmd = ''
 +                traceback.print_exc()
 +                self.check_windows_encoding()
 +            line = self.lastcmd + '\n'
 +        else:
 +            self.lastcmd = self.stdin.readline()
 +            line = self.lastcmd
 +            if not len(line):
 +                raise EOFError
 +        self.lineno += 1
 +        return line
 +
 +    def use_stdin_reader(self, until='', prompt=''):
 +        until += '\n'
 +        while True:
 +            try:
 +                newline = self.get_input_line(prompt=prompt)
 +            except EOFError:
 +                return
 +            if newline == until:
 +                return
 +            yield newline
 +
 +    def cmdloop(self):
 +        """
 +        Adapted from cmd.Cmd's version, because there is literally no way with
 +        cmd.Cmd.cmdloop() to tell the difference between "EOF" showing up in
 +        input and an actual EOF.
 +        """
 +        with self.prepare_loop():
 +            while not self.stop:
 +                try:
 +                    if self.single_statement:
 +                        line = self.single_statement
 +                        self.stop = True
 +                    else:
 +                        line = self.get_input_line(self.prompt)
 +                    self.statement.write(line)
 +                    if self.onecmd(self.statement.getvalue()):
 +                        self.reset_statement()
 +                except EOFError:
 +                    self.handle_eof()
 +                except CQL_ERRORS, cqlerr:
 +                    self.printerr(str(cqlerr))
 +                except KeyboardInterrupt:
 +                    self.reset_statement()
 +                    print
 +
 +    def onecmd(self, statementtext):
 +        """
 +        Returns true if the statement is complete and was handled (meaning it
 +        can be reset).
 +        """
 +
 +        try:
 +            statements, endtoken_escaped = cqlruleset.cql_split_statements(statementtext)
 +        except pylexotron.LexingError, e:
 +            if self.show_line_nums:
 +                self.printerr('Invalid syntax at char %d' % (e.charnum,))
 +            else:
 +                self.printerr('Invalid syntax at line %d, char %d'
 +                              % (e.linenum, e.charnum))
 +            statementline = statementtext.split('\n')[e.linenum - 1]
 +            self.printerr('  %s' % statementline)
 +            self.printerr(' %s^' % (' ' * e.charnum))
 +            return True
 +
 +        while statements and not statements[-1]:
 +            statements = statements[:-1]
 +        if not statements:
 +            return True
 +        if endtoken_escaped or statements[-1][-1][0] != 'endtoken':
 +            self.set_continue_prompt()
 +            return
 +        for st in statements:
 +            try:
 +                self.handle_statement(st, statementtext)
 +            except Exception, e:
 +                if self.debug:
 +                    traceback.print_exc()
 +                else:
 +                    self.printerr(e)
 +        return True
 +
 +    def handle_eof(self):
 +        if self.tty:
 +            print
 +        statement = self.statement.getvalue()
 +        if statement.strip():
 +            if not self.onecmd(statement):
 +                self.printerr('Incomplete statement at end of file')
 +        self.do_exit()
 +
 +    def handle_statement(self, tokens, srcstr):
 +        # Concat multi-line statements and insert into history
 +        if readline is not None:
 +            nl_count = srcstr.count("\n")
 +
 +            new_hist = srcstr.replace("\n", " ").rstrip()
 +
 +            if nl_count > 1 and self.last_hist != new_hist:
 +                readline.add_history(new_hist.encode(self.encoding))
 +
 +            self.last_hist = new_hist
 +        cmdword = tokens[0][1]
 +        if cmdword == '?':
 +            cmdword = 'help'
 +        custom_handler = getattr(self, 'do_' + cmdword.lower(), None)
 +        if custom_handler:
 +            parsed = cqlruleset.cql_whole_parse_tokens(tokens, srcstr=srcstr,
 +                                                       startsymbol='cqlshCommand')
 +            if parsed and not parsed.remainder:
 +                # successful complete parse
 +                return custom_handler(parsed)
 +            else:
 +                return self.handle_parse_error(cmdword, tokens, parsed, srcstr)
 +        return self.perform_statement(cqlruleset.cql_extract_orig(tokens, srcstr))
 +
 +    def handle_parse_error(self, cmdword, tokens, parsed, srcstr):
 +        if cmdword.lower() in ('select', 'insert', 'update', 'delete', 'truncate',
 +                               'create', 'drop', 'alter', 'grant', 'revoke',
 +                               'batch', 'list'):
 +            # hey, maybe they know about some new syntax we don't. type
 +            # assumptions won't work, but maybe the query will.
 +            return self.perform_statement(cqlruleset.cql_extract_orig(tokens, srcstr))
 +        if parsed:
 +            self.printerr('Improper %s command (problem at %r).' % (cmdword, parsed.remainder[0]))
 +        else:
 +            self.printerr('Improper %s command.' % cmdword)
 +
 +    def do_use(self, parsed):
 +        ksname = parsed.get_binding('ksname')
 +        success, _ = self.perform_simple_statement(SimpleStatement(parsed.extract_orig()))
 +        if success:
 +            if ksname[0] == '"' and ksname[-1] == '"':
 +                self.current_keyspace = self.cql_unprotect_name(ksname)
 +            else:
 +                self.current_keyspace = ksname.lower()
 +
 +    def do_select(self, parsed):
 +        tracing_was_enabled = self.tracing_enabled
 +        ksname = parsed.get_binding('ksname')
 +        stop_tracing = ksname == 'system_traces' or (ksname is None and self.current_keyspace == 'system_traces')
 +        self.tracing_enabled = self.tracing_enabled and not stop_tracing
 +        statement = parsed.extract_orig()
 +        self.perform_statement(statement)
 +        self.tracing_enabled = tracing_was_enabled
 +
 +    def perform_statement(self, statement):
 +        stmt = SimpleStatement(statement, consistency_level=self.consistency_level, serial_consistency_level=self.serial_consistency_level, fetch_size=self.page_size if self.use_paging else None)
 +        success, future = self.perform_simple_statement(stmt)
 +
 +        if future:
 +            if future.warnings:
 +                self.print_warnings(future.warnings)
 +
 +            if self.tracing_enabled:
 +                try:
 +                    for trace in future.get_all_query_traces(self.max_trace_wait):
 +                        print_trace(self, trace)
 +                except TraceUnavailable:
 +                    msg = "Statement trace did not complete within %d seconds; trace data may be incomplete." % (self.session.max_trace_wait,)
 +                    self.writeresult(msg, color=RED)
 +                    for trace_id in future.get_query_trace_ids():
 +                        self.show_session(trace_id, partial_session=True)
 +                except Exception, err:
 +                    self.printerr("Unable to fetch query trace: %s" % (str(err),))
 +
 +        return success
 +
 +    def parse_for_table_meta(self, query_string):
 +        try:
 +            parsed = cqlruleset.cql_parse(query_string)[1]
 +        except IndexError:
 +            return None
 +        ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +        cf = self.cql_unprotect_name(parsed.get_binding('cfname'))
 +        return self.get_table_meta(ks, cf)
 +
 +    def perform_simple_statement(self, statement):
 +        if not statement:
 +            return False, None
 +
 +        while True:
 +            try:
 +                future = self.session.execute_async(statement, trace=self.tracing_enabled)
 +                result = future.result()
 +                break
 +            except cassandra.OperationTimedOut, err:
 +                self.refresh_schema_metadata_best_effort()
 +                self.printerr(str(err.__class__.__name__) + ": " + str(err))
 +                return False, None
 +            except CQL_ERRORS, err:
 +                self.printerr(str(err.__class__.__name__) + ": " + str(err))
 +                return False, None
 +            except Exception, err:
 +                import traceback
 +                self.printerr(traceback.format_exc())
 +                return False, None
 +
 +        if statement.query_string[:6].lower() == 'select':
 +            self.print_result(result, self.parse_for_table_meta(statement.query_string))
 +        elif statement.query_string.lower().startswith("list users") or statement.query_string.lower().startswith("list roles"):
 +            self.print_result(result, self.get_table_meta('system_auth', 'roles'))
 +        elif statement.query_string.lower().startswith("list"):
 +            self.print_result(result, self.get_table_meta('system_auth', 'role_permissions'))
 +        elif result:
 +            # CAS INSERT/UPDATE
 +            self.writeresult("")
 +            self.print_static_result(result.column_names, list(result), self.parse_for_table_meta(statement.query_string))
 +        self.flush_output()
 +        return True, future
 +
 +    def print_result(self, result, table_meta):
 +        self.decoding_errors = []
 +
 +        self.writeresult("")
 +        if result.has_more_pages and self.tty:
 +            num_rows = 0
 +            while True:
 +                page = result.current_rows
 +                if page:
 +                    num_rows += len(page)
 +                    self.print_static_result(result.column_names, page, table_meta)
 +                if result.has_more_pages:
 +                    raw_input("---MORE---")
 +                    result.fetch_next_page()
 +                else:
 +                    break
 +        else:
 +            rows = list(result)
 +            num_rows = len(rows)
 +            self.print_static_result(result.column_names, rows, table_meta)
 +        self.writeresult("(%d rows)" % num_rows)
 +
 +        if self.decoding_errors:
 +            for err in self.decoding_errors[:2]:
 +                self.writeresult(err.message(), color=RED)
 +            if len(self.decoding_errors) > 2:
 +                self.writeresult('%d more decoding errors suppressed.'
 +                                 % (len(self.decoding_errors) - 2), color=RED)
 +
 +    def print_static_result(self, column_names, rows, table_meta):
 +        if not column_names and not table_meta:
 +            return
 +        column_names = column_names or table_meta.columns.keys()
 +        formatted_names = [self.myformat_colname(name, table_meta) for name in column_names]
 +        formatted_values = [map(self.myformat_value, row.values()) for row in rows]
 +
 +        if self.expand_enabled:
 +            self.print_formatted_result_vertically(formatted_names, formatted_values)
 +        else:
 +            self.print_formatted_result(formatted_names, formatted_values)
 +
 +    def print_formatted_result(self, formatted_names, formatted_values):
 +        # determine column widths
 +        widths = [n.displaywidth for n in formatted_names]
 +        if formatted_values is not None:
 +            for fmtrow in formatted_values:
 +                for num, col in enumerate(fmtrow):
 +                    widths[num] = max(widths[num], col.displaywidth)
 +
 +        # print header
 +        header = ' | '.join(hdr.ljust(w, color=self.color) for (hdr, w) in zip(formatted_names, widths))
 +        self.writeresult(' ' + header.rstrip())
 +        self.writeresult('-%s-' % '-+-'.join('-' * w for w in widths))
 +
 +        # stop if there are no rows
 +        if formatted_values is None:
 +            self.writeresult("")
 +            return
 +
 +        # print row data
 +        for row in formatted_values:
 +            line = ' | '.join(col.rjust(w, color=self.color) for (col, w) in zip(row, widths))
 +            self.writeresult(' ' + line)
 +
 +        self.writeresult("")
 +
 +    def print_formatted_result_vertically(self, formatted_names, formatted_values):
 +        max_col_width = max([n.displaywidth for n in formatted_names])
 +        max_val_width = max([n.displaywidth for row in formatted_values for n in row])
 +
 +        # for each row returned, list all the column-value pairs
 +        for row_id, row in enumerate(formatted_values):
 +            self.writeresult("@ Row %d" % (row_id + 1))
 +            self.writeresult('-%s-' % '-+-'.join(['-' * max_col_width, '-' * max_val_width]))
 +            for field_id, field in enumerate(row):
 +                column = formatted_names[field_id].ljust(max_col_width, color=self.color)
 +                value = field.ljust(field.displaywidth, color=self.color)
 +                self.writeresult(' ' + " | ".join([column, value]))
 +            self.writeresult('')
 +
 +    def print_warnings(self, warnings):
 +        if warnings is None or len(warnings) == 0:
 +            return
 +
 +        self.writeresult('')
 +        self.writeresult('Warnings :')
 +        for warning in warnings:
 +            self.writeresult(warning)
 +            self.writeresult('')
 +
 +    def emptyline(self):
 +        pass
 +
 +    def parseline(self, line):
 +        # this shouldn't be needed
 +        raise NotImplementedError
 +
 +    def complete(self, text, state):
 +        if readline is None:
 +            return
 +        if state == 0:
 +            try:
 +                self.completion_matches = self.find_completions(text)
 +            except Exception:
 +                if debug_completion:
 +                    import traceback
 +                    traceback.print_exc()
 +                else:
 +                    raise
 +        try:
 +            return self.completion_matches[state]
 +        except IndexError:
 +            return None
 +
 +    def find_completions(self, text):
 +        curline = readline.get_line_buffer()
 +        prevlines = self.statement.getvalue()
 +        wholestmt = prevlines + curline
 +        begidx = readline.get_begidx() + len(prevlines)
 +        stuff_to_complete = wholestmt[:begidx]
 +        return cqlruleset.cql_complete(stuff_to_complete, text, cassandra_conn=self,
 +                                       debug=debug_completion, startsymbol='cqlshCommand')
 +
 +    def set_prompt(self, prompt, prepend_user=False):
 +        if prepend_user and self.username:
 +            self.prompt = "%s@%s" % (self.username, prompt)
 +            return
 +        self.prompt = prompt
 +
 +    def cql_unprotect_name(self, namestr):
 +        if namestr is None:
 +            return
 +        return cqlruleset.dequote_name(namestr)
 +
 +    def cql_unprotect_value(self, valstr):
 +        if valstr is not None:
 +            return cqlruleset.dequote_value(valstr)
 +
 +    def print_recreate_keyspace(self, ksdef, out):
 +        out.write(ksdef.export_as_string())
 +        out.write("\n")
 +
 +    def print_recreate_columnfamily(self, ksname, cfname, out):
 +        """
 +        Output CQL commands which should be pasteable back into a CQL session
 +        to recreate the given table.
 +
 +        Writes output to the given out stream.
 +        """
 +        out.write(self.get_table_meta(ksname, cfname).export_as_string())
 +        out.write("\n")
 +
 +    def print_recreate_index(self, ksname, idxname, out):
 +        """
 +        Output CQL commands which should be pasteable back into a CQL session
 +        to recreate the given index.
 +
 +        Writes output to the given out stream.
 +        """
 +        out.write(self.get_index_meta(ksname, idxname).export_as_string())
 +        out.write("\n")
 +
 +    def print_recreate_object(self, ks, name, out):
 +        """
 +        Output CQL commands which should be pasteable back into a CQL session
 +        to recreate the given object (ks, table or index).
 +
 +        Writes output to the given out stream.
 +        """
 +        out.write(self.get_object_meta(ks, name).export_as_string())
 +        out.write("\n")
 +
 +    def describe_keyspaces(self):
 +        print
 +        cmd.Cmd.columnize(self, protect_names(self.get_keyspace_names()))
 +        print
 +
 +    def describe_keyspace(self, ksname):
 +        print
 +        self.print_recreate_keyspace(self.get_keyspace_meta(ksname), sys.stdout)
 +        print
 +
 +    def describe_columnfamily(self, ksname, cfname):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        if ksname is None:
 +            raise NoKeyspaceError("No keyspace specified and no current keyspace")
 +        print
 +        self.print_recreate_columnfamily(ksname, cfname, sys.stdout)
 +        print
 +
 +    def describe_index(self, ksname, idxname):
 +        print
 +        self.print_recreate_index(ksname, idxname, sys.stdout)
 +        print
 +
 +    def describe_object(self, ks, name):
 +        print
 +        self.print_recreate_object(ks, name, sys.stdout)
 +        print
 +
 +    def describe_columnfamilies(self, ksname):
 +        print
 +        if ksname is None:
 +            for k in self.get_keyspaces():
 +                name = protect_name(k.name)
 +                print 'Keyspace %s' % (name,)
 +                print '---------%s' % ('-' * len(name))
 +                cmd.Cmd.columnize(self, protect_names(self.get_columnfamily_names(k.name)))
 +                print
 +        else:
 +            cmd.Cmd.columnize(self, protect_names(self.get_columnfamily_names(ksname)))
 +            print
 +
 +    def describe_functions(self, ksname):
 +        print
 +        if ksname is None:
 +            for ksmeta in self.get_keyspaces():
 +                name = protect_name(ksmeta.name)
 +                print 'Keyspace %s' % (name,)
 +                print '---------%s' % ('-' * len(name))
 +                cmd.Cmd.columnize(self, protect_names(ksmeta.functions.keys()))
 +                print
 +        else:
 +            ksmeta = self.get_keyspace_meta(ksname)
 +            cmd.Cmd.columnize(self, protect_names(ksmeta.functions.keys()))
 +            print
 +
 +    def describe_function(self, ksname, functionname):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        if ksname is None:
 +            raise NoKeyspaceError("No keyspace specified and no current keyspace")
 +        print
 +        ksmeta = self.get_keyspace_meta(ksname)
 +        functions = filter(lambda f: f.name == functionname, ksmeta.functions.values())
 +        if len(functions) == 0:
 +            raise FunctionNotFound("User defined function %r not found" % functionname)
 +        print "\n\n".join(func.as_cql_query(formatted=True) for func in functions)
 +        print
 +
 +    def describe_aggregates(self, ksname):
 +        print
 +        if ksname is None:
 +            for ksmeta in self.get_keyspaces():
 +                name = protect_name(ksmeta.name)
 +                print 'Keyspace %s' % (name,)
 +                print '---------%s' % ('-' * len(name))
 +                cmd.Cmd.columnize(self, protect_names(ksmeta.aggregates.keys()))
 +                print
 +        else:
 +            ksmeta = self.get_keyspace_meta(ksname)
 +            cmd.Cmd.columnize(self, protect_names(ksmeta.aggregates.keys()))
 +            print
 +
 +    def describe_aggregate(self, ksname, aggregatename):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        if ksname is None:
 +            raise NoKeyspaceError("No keyspace specified and no current keyspace")
 +        print
 +        ksmeta = self.get_keyspace_meta(ksname)
 +        aggregates = filter(lambda f: f.name == aggregatename, ksmeta.aggregates.values())
 +        if len(aggregates) == 0:
 +            raise FunctionNotFound("User defined aggregate %r not found" % aggregatename)
 +        print "\n\n".join(aggr.as_cql_query(formatted=True) for aggr in aggregates)
 +        print
 +
 +    def describe_usertypes(self, ksname):
 +        print
 +        if ksname is None:
 +            for ksmeta in self.get_keyspaces():
 +                name = protect_name(ksmeta.name)
 +                print 'Keyspace %s' % (name,)
 +                print '---------%s' % ('-' * len(name))
 +                cmd.Cmd.columnize(self, protect_names(ksmeta.user_types.keys()))
 +                print
 +        else:
 +            ksmeta = self.get_keyspace_meta(ksname)
 +            cmd.Cmd.columnize(self, protect_names(ksmeta.user_types.keys()))
 +            print
 +
 +    def describe_usertype(self, ksname, typename):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        if ksname is None:
 +            raise NoKeyspaceError("No keyspace specified and no current keyspace")
 +        print
 +        ksmeta = self.get_keyspace_meta(ksname)
 +        try:
 +            usertype = ksmeta.user_types[typename]
 +        except KeyError:
 +            raise UserTypeNotFound("User type %r not found" % typename)
 +        print usertype.as_cql_query(formatted=True)
 +        print
 +
 +    def describe_cluster(self):
 +        print '\nCluster: %s' % self.get_cluster_name()
 +        p = trim_if_present(self.get_partitioner(), 'org.apache.cassandra.dht.')
 +        print 'Partitioner: %s\n' % p
 +        # TODO: snitch?
 +        # snitch = trim_if_present(self.get_snitch(), 'org.apache.cassandra.locator.')
 +        # print 'Snitch: %s\n' % snitch
 +        if self.current_keyspace is not None and self.current_keyspace != 'system':
 +            print "Range ownership:"
 +            ring = self.get_ring(self.current_keyspace)
 +            for entry in ring.items():
 +                print ' %39s  [%s]' % (str(entry[0].value), ', '.join([host.address for host in entry[1]]))
 +            print
 +
 +    def describe_schema(self, include_system=False):
 +        print
 +        for k in self.get_keyspaces():
 +            if include_system or k.name not in cql3handling.SYSTEM_KEYSPACES:
 +                self.print_recreate_keyspace(k, sys.stdout)
 +                print
 +
 +    def do_describe(self, parsed):
 +        """
 +        DESCRIBE [cqlsh only]
 +
 +        (DESC may be used as a shorthand.)
 +
 +          Outputs information about the connected Cassandra cluster, or about
 +          the data objects stored in the cluster. Use in one of the following ways:
 +
 +        DESCRIBE KEYSPACES
 +
 +          Output the names of all keyspaces.
 +
 +        DESCRIBE KEYSPACE [<keyspacename>]
 +
 +          Output CQL commands that could be used to recreate the given keyspace,
 +          and the objects in it (such as tables, types, functions, etc.).
 +          In some cases, as the CQL interface matures, there will be some metadata
 +          about a keyspace that is not representable with CQL. That metadata will not be shown.
 +
 +          The '<keyspacename>' argument may be omitted, in which case the current
 +          keyspace will be described.
 +
 +        DESCRIBE TABLES
 +
 +          Output the names of all tables in the current keyspace, or in all
 +          keyspaces if there is no current keyspace.
 +
 +        DESCRIBE TABLE [<keyspace>.]<tablename>
 +
 +          Output CQL commands that could be used to recreate the given table.
 +          In some cases, as above, there may be table metadata which is not
 +          representable and which will not be shown.
 +
 +        DESCRIBE INDEX <indexname>
 +
 +          Output the CQL command that could be used to recreate the given index.
 +          In some cases, there may be index metadata which is not representable
 +          and which will not be shown.
 +
 +        DESCRIBE CLUSTER
 +
 +          Output information about the connected Cassandra cluster, such as the
 +          cluster name, and the partitioner and snitch in use. When you are
 +          connected to a non-system keyspace, also shows endpoint-range
 +          ownership information for the Cassandra ring.
 +
 +        DESCRIBE [FULL] SCHEMA
 +
 +          Output CQL commands that could be used to recreate the entire (non-system) schema.
 +          Works as though "DESCRIBE KEYSPACE k" was invoked for each non-system keyspace
 +          k. Use DESCRIBE FULL SCHEMA to include the system keyspaces.
 +
 +        DESCRIBE TYPES
 +
 +          Output the names of all user-defined-types in the current keyspace, or in all
 +          keyspaces if there is no current keyspace.
 +
 +        DESCRIBE TYPE [<keyspace>.]<type>
 +
 +          Output the CQL command that could be used to recreate the given user-defined-type.
 +
 +        DESCRIBE FUNCTIONS
 +
 +          Output the names of all user-defined-functions in the current keyspace, or in all
 +          keyspaces if there is no current keyspace.
 +
 +        DESCRIBE FUNCTION [<keyspace>.]<function>
 +
 +          Output the CQL command that could be used to recreate the given user-defined-function.
 +
 +        DESCRIBE AGGREGATES
 +
 +          Output the names of all user-defined-aggregates in the current keyspace, or in all
 +          keyspaces if there is no current keyspace.
 +
 +        DESCRIBE AGGREGATE [<keyspace>.]<aggregate>
 +
 +          Output the CQL command that could be used to recreate the given user-defined-aggregate.
 +
 +        DESCRIBE <objname>
 +
 +          Output CQL commands that could be used to recreate the entire object schema,
 +          where object can be either a keyspace or a table or an index (in this order).
 +  """
 +        what = parsed.matched[1][1].lower()
 +        if what == 'functions':
 +            self.describe_functions(self.current_keyspace)
 +        elif what == 'function':
 +            ksname = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            functionname = self.cql_unprotect_name(parsed.get_binding('udfname'))
 +            self.describe_function(ksname, functionname)
 +        elif what == 'aggregates':
 +            self.describe_aggregates(self.current_keyspace)
 +        elif what == 'aggregate':
 +            ksname = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            aggregatename = self.cql_unprotect_name(parsed.get_binding('udaname'))
 +            self.describe_aggregate(ksname, aggregatename)
 +        elif what == 'keyspaces':
 +            self.describe_keyspaces()
 +        elif what == 'keyspace':
 +            ksname = self.cql_unprotect_name(parsed.get_binding('ksname', ''))
 +            if not ksname:
 +                ksname = self.current_keyspace
 +                if ksname is None:
 +                    self.printerr('Not in any keyspace.')
 +                    return
 +            self.describe_keyspace(ksname)
 +        elif what in ('columnfamily', 'table'):
 +            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            cf = self.cql_unprotect_name(parsed.get_binding('cfname'))
 +            self.describe_columnfamily(ks, cf)
 +        elif what == 'index':
 +            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            idx = self.cql_unprotect_name(parsed.get_binding('idxname', None))
 +            self.describe_index(ks, idx)
 +        elif what in ('columnfamilies', 'tables'):
 +            self.describe_columnfamilies(self.current_keyspace)
 +        elif what == 'types':
 +            self.describe_usertypes(self.current_keyspace)
 +        elif what == 'type':
 +            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            ut = self.cql_unprotect_name(parsed.get_binding('utname'))
 +            self.describe_usertype(ks, ut)
 +        elif what == 'cluster':
 +            self.describe_cluster()
 +        elif what == 'schema':
 +            self.describe_schema(False)
 +        elif what == 'full' and parsed.matched[2][1].lower() == 'schema':
 +            self.describe_schema(True)
 +        elif what:
 +            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            name = self.cql_unprotect_name(parsed.get_binding('cfname'))
 +            if not name:
 +                name = self.cql_unprotect_name(parsed.get_binding('idxname', None))
 +            self.describe_object(ks, name)
 +    do_desc = do_describe
 +
 +    def do_copy(self, parsed):
 +        r"""
 +        COPY [cqlsh only]
 +
 +          COPY x FROM: Imports CSV data into a Cassandra table
 +          COPY x TO: Exports data from a Cassandra table in CSV format.
 +
 +        COPY <table_name> [ ( column [, ...] ) ]
 +             FROM ( '<file_pattern_1, file_pattern_2, ... file_pattern_n>' | STDIN )
 +             [ WITH <option>='value' [AND ...] ];
 +
 +        File patterns are either file names or valid python glob expressions, e.g. *.csv or folder/*.csv.
 +
 +        COPY <table_name> [ ( column [, ...] ) ]
 +             TO ( '<filename>' | STDOUT )
 +             [ WITH <option>='value' [AND ...] ];
 +
 +        Available common COPY options and defaults:
 +
 +          DELIMITER=','           - character that appears between records
 +          QUOTE='"'               - quoting character to be used to quote fields
 +          ESCAPE='\'              - character to appear before the QUOTE char when quoted
 +          HEADER=false            - whether to ignore the first line
 +          NULL=''                 - string that represents a null value
 +          DATETIMEFORMAT=         - timestamp strftime format
 +            '%Y-%m-%d %H:%M:%S%z'   defaults to time_format value in cqlshrc
 +          MAXATTEMPTS=5           - the maximum number of attempts per batch or range
 +          REPORTFREQUENCY=0.25    - the frequency with which we display status updates in seconds
 +          DECIMALSEP='.'          - the separator for decimal values
 +          THOUSANDSSEP=''         - the separator for thousands digit groups
 +          BOOLSTYLE='True,False'  - the representation for booleans, case insensitive, specify true followed by false,
 +                                    for example yes,no or 1,0
 +          NUMPROCESSES=n          - the number of worker processes, by default the number of cores minus one
 +                                    capped at 16
 +          CONFIGFILE=''           - a configuration file with the same format as .cqlshrc (see the Python ConfigParser
 +                                    documentation) where you can specify WITH options under the following optional
 +                                    sections: [copy], [copy-to], [copy-from], [copy:ks.table], [copy-to:ks.table],
 +                                    [copy-from:ks.table], where <ks> is your keyspace name and <table> is your table
 +                                    name. Options are read from these sections, in the order specified
 +                                    above, and command line options always override options in configuration files.
 +                                    Depending on the COPY direction, only the relevant copy-from or copy-to sections
 +                                    are used. If no configfile is specified then .cqlshrc is searched instead.
 +          RATEFILE=''             - an optional file where to print the output statistics
 +
 +        Available COPY FROM options and defaults:
 +
-           CHUNKSIZE=1000          - the size of chunks passed to worker processes
++          CHUNKSIZE=5000          - the size of chunks passed to worker processes
 +          INGESTRATE=100000       - an approximate ingest rate in rows per second
-           MINBATCHSIZE=2          - the minimum size of an import batch
++          MINBATCHSIZE=10         - the minimum size of an import batch
 +          MAXBATCHSIZE=20         - the maximum size of an import batch
 +          MAXROWS=-1              - the maximum number of rows, -1 means no maximum
 +          SKIPROWS=0              - the number of rows to skip
 +          SKIPCOLS=''             - a comma separated list of column names to skip
 +          MAXPARSEERRORS=-1       - the maximum global number of parsing errors, -1 means no maximum
 +          MAXINSERTERRORS=-1      - the maximum global number of insert errors, -1 means no maximum
 +          ERRFILE=''              - a file where to store all rows that could not be imported, by default this is
 +                                    import_ks_table.err where <ks> is your keyspace and <table> is your table name.
++          PREPAREDSTATEMENTS=True - whether to use prepared statements when importing, by default True. Set this to
++                                    False if you don't mind shifting data parsing to the cluster. The cluster will also
++                                    have to compile every batch statement. For large and oversized clusters
++                                    this will result in a faster import but for smaller clusters it may generate
++                                    timeouts.
 +
 +        Available COPY TO options and defaults:
 +
 +          ENCODING='utf8'          - encoding for CSV output
 +          PAGESIZE='1000'          - the page size for fetching results
 +          PAGETIMEOUT=10           - the page timeout in seconds for fetching results
 +          BEGINTOKEN=''            - the minimum token string to consider when exporting data
 +          ENDTOKEN=''              - the maximum token string to consider when exporting data
 +          MAXREQUESTS=6            - the maximum number of requests each worker process can work on in parallel
 +          MAXOUTPUTSIZE='-1'       - the maximum size of the output file measured in number of lines,
 +                                     beyond this maximum the output file will be split into segments,
 +                                     -1 means unlimited.
 +
 +        When entering CSV data on STDIN, you can use the sequence "\."
 +        on a line by itself to end the data input.
 +        """
 +
 +        ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +        if ks is None:
 +            ks = self.current_keyspace
 +            if ks is None:
 +                raise NoKeyspaceError("Not in any keyspace.")
 +        table = self.cql_unprotect_name(parsed.get_binding('cfname'))
 +        columns = parsed.get_binding('colnames', None)
 +        if columns is not None:
 +            columns = map(self.cql_unprotect_name, columns)
 +        else:
 +            # default to all known columns
 +            columns = self.get_column_names(ks, table)
 +
 +        fname = parsed.get_binding('fname', None)
 +        if fname is not None:
 +            fname = self.cql_unprotect_value(fname)
 +
 +        copyoptnames = map(str.lower, parsed.get_binding('optnames', ()))
 +        copyoptvals = map(self.cql_unprotect_value, parsed.get_binding('optvals', ()))
 +        opts = dict(zip(copyoptnames, copyoptvals))
 +
 +        direction = parsed.get_binding('dir').upper()
 +        if direction == 'FROM':
 +            task = ImportTask(self, ks, table, columns, fname, opts, DEFAULT_PROTOCOL_VERSION, CONFIG_FILE)
 +        elif direction == 'TO':
 +            task = ExportTask(self, ks, table, columns, fname, opts, DEFAULT_PROTOCOL_VERSION, CONFIG_FILE)
 +        else:
 +            raise SyntaxError("Unknown direction %s" % direction)
 +
 +        task.run()
 +
 +    def do_show(self, parsed):
 +        """
 +        SHOW [cqlsh only]
 +
 +          Displays information about the current cqlsh session. Can be called in
 +          the following ways:
 +
 +        SHOW VERSION
 +
 +          Shows the version and build of the connected Cassandra instance, as
 +          well as the versions of the CQL spec and the Thrift protocol that
 +          the connected Cassandra instance understands.
 +
 +        SHOW HOST
 +
 +          Shows where cqlsh is currently connected.
 +
 +        SHOW SESSION <sessionid>
 +
 +          Pretty-prints the requested tracing session.
 +        """
 +        showwhat = parsed.get_binding('what').lower()
 +        if showwhat == 'version':
 +            self.get_connection_versions()
 +            self.show_version()
 +        elif showwhat == 'host':
 +            self.show_host()
 +        elif showwhat.startswith('session'):
 +            session_id = parsed.get_binding('sessionid').lower()
 +            self.show_session(UUID(session_id))
 +        else:
 +            self.printerr('Wait, how do I show %r?' % (showwhat,))
 +
 +    def do_source(self, parsed):
 +        """
 +        SOURCE [cqlsh only]
 +
 +        Executes a file containing CQL statements. Gives the output for each
 +        statement in turn, if any, or any errors that occur along the way.
 +
 +        Errors do NOT abort execution of the CQL source file.
 +
 +        Usage:
 +
 +          SOURCE '<file>';
 +
 +        That is, the path to the file to be executed must be given inside a
 +        string literal. The path is interpreted relative to the current working
 +        directory. The tilde shorthand notation ('~/mydir') is supported for
 +        referring to $HOME.
 +
 +        See also the --file option to cqlsh.
 +        """
 +        fname = parsed.get_binding('fname')
 +        fname = os.path.expanduser(self.cql_unprotect_value(fname))
 +        try:
 +            encoding, bom_size = get_file_encoding_bomsize(fname)
 +            f = codecs.open(fname, 'r', encoding)
 +            f.seek(bom_size)
 +        except IOError, e:
 +            self.printerr('Could not open %r: %s' % (fname, e))
 +            return
 +        subshell = Shell(self.hostname, self.port,
 +                         color=self.color, encoding=self.encoding, stdin=f,
 +                         tty=False, use_conn=self.conn, cqlver=self.cql_version,
 +                         display_timestamp_format=self.display_timestamp_format,
 +                         display_date_format=self.display_date_format,
 +                         display_nanotime_format=self.display_nanotime_format,
 +                         display_float_precision=self.display_float_precision,
 +                         max_trace_wait=self.max_trace_wait)
 +        subshell.cmdloop()
 +        f.close()
 +
 +    def do_capture(self, parsed):
 +        """
 +        CAPTURE [cqlsh only]
 +
 +        Begins capturing command output and appending it to a specified file.
 +        Output will not be shown at the console while it is captured.
 +
 +        Usage:
 +
 +          CAPTURE '<file>';
 +          CAPTURE OFF;
 +          CAPTURE;
 +
 +        That is, the path to the file to be appended to must be given inside a
 +        string literal. The path is interpreted relative to the current working
 +        directory. The tilde shorthand notation ('~/mydir') is supported for
 +        referring to $HOME.
 +
 +        Only query result output is captured. Errors and output from cqlsh-only
 +        commands will still be shown in the cqlsh session.
 +
 +        To stop capturing output and show it in the cqlsh session again, use
 +        CAPTURE OFF.
 +
 +        To inspect the current capture configuration, use CAPTURE with no
 +        arguments.
 +        """
 +        fname = parsed.get_binding('fname')
 +        if fname is None:
 +            if self.shunted_query_out is not None:
 +                print "Currently capturing query output to %r." % (self.query_out.name,)
 +            else:
 +                print "Currently not capturing query output."
 +            return
 +
 +        if fname.upper() == 'OFF':
 +            if self.shunted_query_out is None:
 +                self.printerr('Not currently capturing output.')
 +                return
 +            self.query_out.close()
 +            self.query_out = self.shunted_query_out
 +            self.color = self.shunted_color
 +            self.shunted_query_out = None
 +            del self.shunted_color
 +            return
 +
 +        if self.shunted_query_out is not None:
 +            self.printerr('Already capturing output to %s. Use CAPTURE OFF'
 +                          ' to disable.' % (self.query_out.name,))
 +            return
 +
 +        fname = os.path.expanduser(self.cql_unprotect_value(fname))
 +        try:
 +            f = open(fname, 'a')
 +        except IOError, e:
 +            self.printerr('Could not open %r for append: %s' % (fname, e))
 +            return
 +        self.shunted_query_out = self.query_out
 +        self.shunted_color = self.color
 +        self.query_out = f
 +        self.color = False
 +        print 'Now capturing query output to %r.' % (fname,)
 +
 +    def do_tracing(self, parsed):
 +        """
 +        TRACING [cqlsh]
 +
 +          Enables or disables request tracing.
 +
 +        TRACING ON
 +
 +          Enables tracing for all further requests.
 +
 +        TRACING OFF
 +
 +          Disables tracing.
 +
 +        TRACING
 +
 +          TRACING with no arguments shows the current tracing status.
 +        """
 +        self.tracing_enabled = SwitchCommand("TRACING", "Tracing").execute(self.tracing_enabled, parsed, self.printerr)
 +
 +    def do_expand(self, parsed):
 +        """
 +        EXPAND [cqlsh]
 +
 +          Enables or disables expanded (vertical) output.
 +
 +        EXPAND ON
 +
 +          Enables expanded (vertical) output.
 +
 +        EXPAND OFF
 +
 +          Disables expanded (vertical) output.
 +
 +        EXPAND
 +
 +          EXPAND with no arguments shows the current value of expand setting.
 +        """
 +        self.expand_enabled = SwitchCommand("EXPAND", "Expanded output").execute(self.expand_enabled, parsed, self.printerr)
 +
 +    def do_consistency(self, parsed):
 +        """
 +        CONSISTENCY [cqlsh only]
 +
 +           Overrides default consistency level (default level is ONE).
 +
 +        CONSISTENCY <level>
 +
 +           Sets consistency level for future requests.
 +
 +           Valid consistency levels:
 +
 +           ANY, ONE, TWO, THREE, QUORUM, ALL, LOCAL_ONE, LOCAL_QUORUM, EACH_QUORUM, SERIAL and LOCAL_SERIAL.
 +
 +           SERIAL and LOCAL_SERIAL may be used only for SELECTs; will be rejected with updates.
 +
 +        CONSISTENCY
 +
 +           CONSISTENCY with no arguments shows the current consistency level.
 +        """
 +        level = parsed.get_binding('level')
 +        if level is None:
 +            print 'Current consistency level is %s.' % (cassandra.ConsistencyLevel.value_to_name[self.consistency_level])
 +            return
 +
 +        self.consistency_level = cassandra.ConsistencyLevel.name_to_value[level.upper()]
 +        print 'Consistency level set to %s.' % (level.upper(),)
 +
 +    def do_serial(self, parsed):
 +        """
 +        SERIAL CONSISTENCY [cqlsh only]
 +
 +           Overrides serial consistency level (default level is SERIAL).
 +
 +        SERIAL CONSISTENCY <level>
 +
 +           Sets consistency level for future conditional updates.
 +
 +           Valid consistency levels:
 +
 +           SERIAL, LOCAL_SERIAL.
 +
 +        SERIAL CONSISTENCY
 +
 +           SERIAL CONSISTENCY with no arguments shows the current consistency level.
 +        """
 +        level = parsed.get_binding('level')
 +        if level is None:
 +            print 'Current serial consistency level is %s.' % (cassandra.ConsistencyLevel.value_to_name[self.serial_consistency_level])
 +            return
 +
 +        self.serial_consistency_level = cassandra.ConsistencyLevel.name_to_value[level.upper()]
 +        print 'Serial consistency level set to %s.' % (level.upper(),)
 +
 +    def do_login(self, parsed):
 +        """
 +        LOGIN [cqlsh only]
 +
 +           Changes login information without requiring restart.
 +
 +        LOGIN <username> (<password>)
 +
 +           Login using the specified username. If password is specified, it will be used
 +           otherwise, you will be prompted to enter.
 +        """
 +        username = parsed.get_binding('username')
 +        password = parsed.get_binding('password')
 +        if password is None:
 +            password = getpass.getpass()
 +        else:
 +            password = password[1:-1]
 +
 +        auth_provider = PlainTextAuthProvider(username=username, password=password)
 +
 +        conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=self.conn.cql_version,
 +                       protocol_version=self.conn.protocol_version,
 +                       auth_provider=auth_provider,
 +                       ssl_options=self.conn.ssl_options,
 +                       load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]),
 +                       control_connection_timeout=self.conn.connect_timeout,
 +                       connect_timeout=self.conn.connect_timeout)
 +
 +        if self.current_keyspace:
 +            session = conn.connect(self.current_keyspace)
 +        else:
 +            session = conn.connect()
 +
 +        # Update after we've connected in case we fail to authenticate
 +        self.conn = conn
 +        self.auth_provider = auth_provider
 +        self.username = username
 +        self.session = session
 +
 +    def do_exit(self, parsed=None):
 +        """
 +        EXIT/QUIT [cqlsh only]
 +
 +        Exits cqlsh.
 +        """
 +        self.stop = True
 +        if self.owns_connection:
 +            self.conn.shutdown()
 +    do_quit = do_exit
 +
 +    def do_clear(self, parsed):
 +        """
 +        CLEAR/CLS [cqlsh only]
 +
 +        Clears the console.
 +        """
 +        import subprocess
 +        subprocess.call(['clear', 'cls'][is_win], shell=True)
 +    do_cls = do_clear
 +
 +    def do_debug(self, parsed):
 +        import pdb
 +        pdb.set_trace()
 +
 +    def get_help_topics(self):
 +        topics = [t[3:] for t in dir(self) if t.startswith('do_') and getattr(self, t, None).__doc__]
 +        for hide_from_help in ('quit',):
 +            topics.remove(hide_from_help)
 +        return topi

<TRUNCATED>

[13/23] cassandra git commit: Merge commit 'c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc' into cassandra-2.2

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b74ffeaf/bin/cqlsh.py
----------------------------------------------------------------------
diff --cc bin/cqlsh.py
index b3ec1ac,0000000..9082d72
mode 100644,000000..100644
--- a/bin/cqlsh.py
+++ b/bin/cqlsh.py
@@@ -1,2546 -1,0 +1,2566 @@@
 +#!/bin/sh
 +# -*- mode: Python -*-
 +
 +# Licensed to the Apache Software Foundation (ASF) under one
 +# or more contributor license agreements.  See the NOTICE file
 +# distributed with this work for additional information
 +# regarding copyright ownership.  The ASF licenses this file
 +# to you under the Apache License, Version 2.0 (the
 +# "License"); you may not use this file except in compliance
 +# with the License.  You may obtain a copy of the License at
 +#
 +#     http://www.apache.org/licenses/LICENSE-2.0
 +#
 +# Unless required by applicable law or agreed to in writing, software
 +# distributed under the License is distributed on an "AS IS" BASIS,
 +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 +# See the License for the specific language governing permissions and
 +# limitations under the License.
 +
 +""":"
 +# bash code here; finds a suitable python interpreter and execs this file.
 +# prefer unqualified "python" if suitable:
 +python -c 'import sys; sys.exit(not (0x020700b0 < sys.hexversion < 0x03000000))' 2>/dev/null \
 +    && exec python "$0" "$@"
 +for pyver in 2.7; do
 +    which python$pyver > /dev/null 2>&1 && exec python$pyver "$0" "$@"
 +done
 +echo "No appropriate python interpreter found." >&2
 +exit 1
 +":"""
 +
 +from __future__ import with_statement
 +
 +import cmd
 +import codecs
 +import ConfigParser
 +import csv
 +import getpass
 +import locale
 +import optparse
 +import os
 +import platform
 +import sys
 +import traceback
 +import warnings
 +import webbrowser
 +from StringIO import StringIO
 +from contextlib import contextmanager
 +from glob import glob
 +from uuid import UUID
 +
 +if sys.version_info[0] != 2 or sys.version_info[1] != 7:
 +    sys.exit("\nCQL Shell supports only Python 2.7\n")
 +
 +UTF8 = 'utf-8'
 +CP65001 = 'cp65001'  # Win utf-8 variant
 +
 +description = "CQL Shell for Apache Cassandra"
 +version = "5.0.1"
 +
 +readline = None
 +try:
 +    # check if tty first, cause readline doesn't check, and only cares
 +    # about $TERM. we don't want the funky escape code stuff to be
 +    # output if not a tty.
 +    if sys.stdin.isatty():
 +        import readline
 +except ImportError:
 +    pass
 +
 +CQL_LIB_PREFIX = 'cassandra-driver-internal-only-'
 +
 +CASSANDRA_PATH = os.path.join(os.path.dirname(os.path.realpath(__file__)), '..')
 +CASSANDRA_CQL_HTML_FALLBACK = 'https://cassandra.apache.org/doc/cql3/CQL-2.2.html'
 +
 +if os.path.exists(CASSANDRA_PATH + '/doc/cql3/CQL.html'):
 +    # default location of local CQL.html
 +    CASSANDRA_CQL_HTML = 'file://' + CASSANDRA_PATH + '/doc/cql3/CQL.html'
 +elif os.path.exists('/usr/share/doc/cassandra/CQL.html'):
 +    # fallback to package file
 +    CASSANDRA_CQL_HTML = 'file:///usr/share/doc/cassandra/CQL.html'
 +else:
 +    # fallback to online version
 +    CASSANDRA_CQL_HTML = CASSANDRA_CQL_HTML_FALLBACK
 +
 +# On Linux, the Python webbrowser module uses the 'xdg-open' executable
 +# to open a file/URL. But that only works, if the current session has been
 +# opened from _within_ a desktop environment. I.e. 'xdg-open' will fail,
 +# if the session's been opened via ssh to a remote box.
 +#
 +# Use 'python' to get some information about the detected browsers.
 +# >>> import webbrowser
 +# >>> webbrowser._tryorder
 +# >>> webbrowser._browser
 +#
 +if len(webbrowser._tryorder) == 0:
 +    CASSANDRA_CQL_HTML = CASSANDRA_CQL_HTML_FALLBACK
 +elif webbrowser._tryorder[0] == 'xdg-open' and os.environ.get('XDG_DATA_DIRS', '') == '':
 +    # only on Linux (some OS with xdg-open)
 +    webbrowser._tryorder.remove('xdg-open')
 +    webbrowser._tryorder.append('xdg-open')
 +
 +# use bundled libs for python-cql and thrift, if available. if there
 +# is a ../lib dir, use bundled libs there preferentially.
 +ZIPLIB_DIRS = [os.path.join(CASSANDRA_PATH, 'lib')]
 +myplatform = platform.system()
 +is_win = myplatform == 'Windows'
 +
 +# Workaround for supporting CP65001 encoding on python < 3.3 (https://bugs.python.org/issue13216)
 +if is_win and sys.version_info < (3, 3):
 +    codecs.register(lambda name: codecs.lookup(UTF8) if name == CP65001 else None)
 +
 +if myplatform == 'Linux':
 +    ZIPLIB_DIRS.append('/usr/share/cassandra/lib')
 +
 +if os.environ.get('CQLSH_NO_BUNDLED', ''):
 +    ZIPLIB_DIRS = ()
 +
 +
 +def find_zip(libprefix):
 +    for ziplibdir in ZIPLIB_DIRS:
 +        zips = glob(os.path.join(ziplibdir, libprefix + '*.zip'))
 +        if zips:
 +            return max(zips)   # probably the highest version, if multiple
 +
 +cql_zip = find_zip(CQL_LIB_PREFIX)
 +if cql_zip:
 +    ver = os.path.splitext(os.path.basename(cql_zip))[0][len(CQL_LIB_PREFIX):]
 +    sys.path.insert(0, os.path.join(cql_zip, 'cassandra-driver-' + ver))
 +
 +third_parties = ('futures-', 'six-')
 +
 +for lib in third_parties:
 +    lib_zip = find_zip(lib)
 +    if lib_zip:
 +        sys.path.insert(0, lib_zip)
 +
 +warnings.filterwarnings("ignore", r".*blist.*")
 +try:
 +    import cassandra
 +except ImportError, e:
 +    sys.exit("\nPython Cassandra driver not installed, or not on PYTHONPATH.\n"
 +             'You might try "pip install cassandra-driver".\n\n'
 +             'Python: %s\n'
 +             'Module load path: %r\n\n'
 +             'Error: %s\n' % (sys.executable, sys.path, e))
 +
 +from cassandra.auth import PlainTextAuthProvider
 +from cassandra.cluster import Cluster
 +from cassandra.metadata import (ColumnMetadata, KeyspaceMetadata,
 +                                TableMetadata, protect_name, protect_names)
 +from cassandra.policies import WhiteListRoundRobinPolicy
 +from cassandra.query import SimpleStatement, ordered_dict_factory, TraceUnavailable
 +
 +# cqlsh should run correctly when run out of a Cassandra source tree,
 +# out of an unpacked Cassandra tarball, and after a proper package install.
 +cqlshlibdir = os.path.join(CASSANDRA_PATH, 'pylib')
 +if os.path.isdir(cqlshlibdir):
 +    sys.path.insert(0, cqlshlibdir)
 +
 +from cqlshlib import cql3handling, cqlhandling, pylexotron, sslhandling
 +from cqlshlib.copyutil import ExportTask, ImportTask
 +from cqlshlib.displaying import (ANSI_RESET, BLUE, COLUMN_NAME_COLORS, CYAN,
 +                                 RED, FormattedValue, colorme)
 +from cqlshlib.formatting import (DEFAULT_DATE_FORMAT, DEFAULT_NANOTIME_FORMAT,
 +                                 DEFAULT_TIMESTAMP_FORMAT, DateTimeFormat,
 +                                 format_by_type, format_value_utype,
 +                                 formatter_for)
 +from cqlshlib.tracing import print_trace, print_trace_session
 +from cqlshlib.util import get_file_encoding_bomsize, trim_if_present
 +
 +DEFAULT_HOST = '127.0.0.1'
 +DEFAULT_PORT = 9042
 +DEFAULT_CQLVER = '3.3.1'
 +DEFAULT_PROTOCOL_VERSION = 4
 +DEFAULT_CONNECT_TIMEOUT_SECONDS = 5
 +DEFAULT_REQUEST_TIMEOUT_SECONDS = 10
 +
 +DEFAULT_FLOAT_PRECISION = 5
 +DEFAULT_MAX_TRACE_WAIT = 10
 +
 +if readline is not None and readline.__doc__ is not None and 'libedit' in readline.__doc__:
 +    DEFAULT_COMPLETEKEY = '\t'
 +else:
 +    DEFAULT_COMPLETEKEY = 'tab'
 +
 +cqldocs = None
 +cqlruleset = None
 +
 +epilog = """Connects to %(DEFAULT_HOST)s:%(DEFAULT_PORT)d by default. These
 +defaults can be changed by setting $CQLSH_HOST and/or $CQLSH_PORT. When a
 +host (and optional port number) are given on the command line, they take
 +precedence over any defaults.""" % globals()
 +
 +parser = optparse.OptionParser(description=description, epilog=epilog,
 +                               usage="Usage: %prog [options] [host [port]]",
 +                               version='cqlsh ' + version)
 +parser.add_option("-C", "--color", action='store_true', dest='color',
 +                  help='Always use color output')
 +parser.add_option("--no-color", action='store_false', dest='color',
 +                  help='Never use color output')
 +parser.add_option("--browser", dest='browser', help="""The browser to use to display CQL help, where BROWSER can be:
 +                                                    - one of the supported browsers in https://docs.python.org/2/library/webbrowser.html.
 +                                                    - browser path followed by %s, example: /usr/bin/google-chrome-stable %s""")
 +parser.add_option('--ssl', action='store_true', help='Use SSL', default=False)
 +parser.add_option("-u", "--username", help="Authenticate as user.")
 +parser.add_option("-p", "--password", help="Authenticate using password.")
 +parser.add_option('-k', '--keyspace', help='Authenticate to the given keyspace.')
 +parser.add_option("-f", "--file", help="Execute commands from FILE, then exit")
 +parser.add_option('--debug', action='store_true',
 +                  help='Show additional debugging information')
 +parser.add_option("--encoding", help="Specify a non-default encoding for output." +
 +                  " (Default: %s)" % (UTF8,))
 +parser.add_option("--cqlshrc", help="Specify an alternative cqlshrc file location.")
 +parser.add_option('--cqlversion', default=DEFAULT_CQLVER,
 +                  help='Specify a particular CQL version (default: %default).'
 +                       ' Examples: "3.0.3", "3.1.0"')
 +parser.add_option("-e", "--execute", help='Execute the statement and quit.')
 +parser.add_option("--connect-timeout", default=DEFAULT_CONNECT_TIMEOUT_SECONDS, dest='connect_timeout',
 +                  help='Specify the connection timeout in seconds (default: %default seconds).')
 +parser.add_option("--request-timeout", default=DEFAULT_REQUEST_TIMEOUT_SECONDS, dest='request_timeout',
 +                  help='Specify the default request timeout in seconds (default: %default seconds).')
 +parser.add_option("-t", "--tty", action='store_true', dest='tty',
 +                  help='Force tty mode (command prompt).')
 +
 +optvalues = optparse.Values()
 +(options, arguments) = parser.parse_args(sys.argv[1:], values=optvalues)
 +
 +# BEGIN history/config definition
 +HISTORY_DIR = os.path.expanduser(os.path.join('~', '.cassandra'))
 +
 +if hasattr(options, 'cqlshrc'):
 +    CONFIG_FILE = options.cqlshrc
 +    if not os.path.exists(CONFIG_FILE):
 +        print '\nWarning: Specified cqlshrc location `%s` does not exist.  Using `%s` instead.\n' % (CONFIG_FILE, HISTORY_DIR)
 +        CONFIG_FILE = os.path.join(HISTORY_DIR, 'cqlshrc')
 +else:
 +    CONFIG_FILE = os.path.join(HISTORY_DIR, 'cqlshrc')
 +
 +HISTORY = os.path.join(HISTORY_DIR, 'cqlsh_history')
 +if not os.path.exists(HISTORY_DIR):
 +    try:
 +        os.mkdir(HISTORY_DIR)
 +    except OSError:
 +        print '\nWarning: Cannot create directory at `%s`. Command history will not be saved.\n' % HISTORY_DIR
 +
 +OLD_CONFIG_FILE = os.path.expanduser(os.path.join('~', '.cqlshrc'))
 +if os.path.exists(OLD_CONFIG_FILE):
 +    if os.path.exists(CONFIG_FILE):
 +        print '\nWarning: cqlshrc config files were found at both the old location (%s) and \
 +                the new location (%s), the old config file will not be migrated to the new \
 +                location, and the new location will be used for now.  You should manually \
 +                consolidate the config files at the new location and remove the old file.' \
 +                % (OLD_CONFIG_FILE, CONFIG_FILE)
 +    else:
 +        os.rename(OLD_CONFIG_FILE, CONFIG_FILE)
 +OLD_HISTORY = os.path.expanduser(os.path.join('~', '.cqlsh_history'))
 +if os.path.exists(OLD_HISTORY):
 +    os.rename(OLD_HISTORY, HISTORY)
 +# END history/config definition
 +
 +CQL_ERRORS = (
 +    cassandra.AlreadyExists, cassandra.AuthenticationFailed, cassandra.InvalidRequest,
 +    cassandra.Timeout, cassandra.Unauthorized, cassandra.OperationTimedOut,
 +    cassandra.cluster.NoHostAvailable,
 +    cassandra.connection.ConnectionBusy, cassandra.connection.ProtocolError, cassandra.connection.ConnectionException,
 +    cassandra.protocol.ErrorMessage, cassandra.protocol.InternalError, cassandra.query.TraceUnavailable
 +)
 +
 +debug_completion = bool(os.environ.get('CQLSH_DEBUG_COMPLETION', '') == 'YES')
 +
 +# we want the cql parser to understand our cqlsh-specific commands too
 +my_commands_ending_with_newline = (
 +    'help',
 +    '?',
 +    'consistency',
 +    'serial',
 +    'describe',
 +    'desc',
 +    'show',
 +    'source',
 +    'capture',
 +    'login',
 +    'debug',
 +    'tracing',
 +    'expand',
 +    'paging',
 +    'exit',
 +    'quit',
 +    'clear',
 +    'cls'
 +)
 +
 +
 +cqlsh_syntax_completers = []
 +
 +
 +def cqlsh_syntax_completer(rulename, termname):
 +    def registrator(f):
 +        cqlsh_syntax_completers.append((rulename, termname, f))
 +        return f
 +    return registrator
 +
 +
 +cqlsh_extra_syntax_rules = r'''
 +<cqlshCommand> ::= <CQL_Statement>
 +                 | <specialCommand> ( ";" | "\n" )
 +                 ;
 +
 +<specialCommand> ::= <describeCommand>
 +                   | <consistencyCommand>
 +                   | <serialConsistencyCommand>
 +                   | <showCommand>
 +                   | <sourceCommand>
 +                   | <captureCommand>
 +                   | <copyCommand>
 +                   | <loginCommand>
 +                   | <debugCommand>
 +                   | <helpCommand>
 +                   | <tracingCommand>
 +                   | <expandCommand>
 +                   | <exitCommand>
 +                   | <pagingCommand>
 +                   | <clearCommand>
 +                   ;
 +
 +<describeCommand> ::= ( "DESCRIBE" | "DESC" )
 +                                  ( "FUNCTIONS"
 +                                  | "FUNCTION" udf=<anyFunctionName>
 +                                  | "AGGREGATES"
 +                                  | "AGGREGATE" uda=<userAggregateName>
 +                                  | "KEYSPACES"
 +                                  | "KEYSPACE" ksname=<keyspaceName>?
 +                                  | ( "COLUMNFAMILY" | "TABLE" ) cf=<columnFamilyName>
 +                                  | "INDEX" idx=<indexName>
 +                                  | ( "COLUMNFAMILIES" | "TABLES" )
 +                                  | "FULL"? "SCHEMA"
 +                                  | "CLUSTER"
 +                                  | "TYPES"
 +                                  | "TYPE" ut=<userTypeName>
 +                                  | (ksname=<keyspaceName> | cf=<columnFamilyName> | idx=<indexName>))
 +                    ;
 +
 +<consistencyCommand> ::= "CONSISTENCY" ( level=<consistencyLevel> )?
 +                       ;
 +
 +<consistencyLevel> ::= "ANY"
 +                     | "ONE"
 +                     | "TWO"
 +                     | "THREE"
 +                     | "QUORUM"
 +                     | "ALL"
 +                     | "LOCAL_QUORUM"
 +                     | "EACH_QUORUM"
 +                     | "SERIAL"
 +                     | "LOCAL_SERIAL"
 +                     | "LOCAL_ONE"
 +                     ;
 +
 +<serialConsistencyCommand> ::= "SERIAL" "CONSISTENCY" ( level=<serialConsistencyLevel> )?
 +                             ;
 +
 +<serialConsistencyLevel> ::= "SERIAL"
 +                           | "LOCAL_SERIAL"
 +                           ;
 +
 +<showCommand> ::= "SHOW" what=( "VERSION" | "HOST" | "SESSION" sessionid=<uuid> )
 +                ;
 +
 +<sourceCommand> ::= "SOURCE" fname=<stringLiteral>
 +                  ;
 +
 +<captureCommand> ::= "CAPTURE" ( fname=( <stringLiteral> | "OFF" ) )?
 +                   ;
 +
 +<copyCommand> ::= "COPY" cf=<columnFamilyName>
 +                         ( "(" [colnames]=<colname> ( "," [colnames]=<colname> )* ")" )?
 +                         ( dir="FROM" ( fname=<stringLiteral> | "STDIN" )
 +                         | dir="TO"   ( fname=<stringLiteral> | "STDOUT" ) )
 +                         ( "WITH" <copyOption> ( "AND" <copyOption> )* )?
 +                ;
 +
 +<copyOption> ::= [optnames]=(<identifier>|<reserved_identifier>) "=" [optvals]=<copyOptionVal>
 +               ;
 +
 +<copyOptionVal> ::= <identifier>
 +                  | <reserved_identifier>
 +                  | <term>
 +                  ;
 +
 +# avoiding just "DEBUG" so that this rule doesn't get treated as a terminal
 +<debugCommand> ::= "DEBUG" "THINGS"?
 +                 ;
 +
 +<helpCommand> ::= ( "HELP" | "?" ) [topic]=( /[a-z_]*/ )*
 +                ;
 +
 +<tracingCommand> ::= "TRACING" ( switch=( "ON" | "OFF" ) )?
 +                   ;
 +
 +<expandCommand> ::= "EXPAND" ( switch=( "ON" | "OFF" ) )?
 +                   ;
 +
 +<pagingCommand> ::= "PAGING" ( switch=( "ON" | "OFF" | /[0-9]+/) )?
 +                  ;
 +
 +<loginCommand> ::= "LOGIN" username=<username> (password=<stringLiteral>)?
 +                 ;
 +
 +<exitCommand> ::= "exit" | "quit"
 +                ;
 +
 +<clearCommand> ::= "CLEAR" | "CLS"
 +                 ;
 +
 +<qmark> ::= "?" ;
 +'''
 +
 +
 +@cqlsh_syntax_completer('helpCommand', 'topic')
 +def complete_help(ctxt, cqlsh):
 +    return sorted([t.upper() for t in cqldocs.get_help_topics() + cqlsh.get_help_topics()])
 +
 +
 +def complete_source_quoted_filename(ctxt, cqlsh):
 +    partial_path = ctxt.get_binding('partial', '')
 +    head, tail = os.path.split(partial_path)
 +    exhead = os.path.expanduser(head)
 +    try:
 +        contents = os.listdir(exhead or '.')
 +    except OSError:
 +        return ()
 +    matches = filter(lambda f: f.startswith(tail), contents)
 +    annotated = []
 +    for f in matches:
 +        match = os.path.join(head, f)
 +        if os.path.isdir(os.path.join(exhead, f)):
 +            match += '/'
 +        annotated.append(match)
 +    return annotated
 +
 +
 +cqlsh_syntax_completer('sourceCommand', 'fname')(complete_source_quoted_filename)
 +cqlsh_syntax_completer('captureCommand', 'fname')(complete_source_quoted_filename)
 +
 +
 +@cqlsh_syntax_completer('copyCommand', 'fname')
 +def copy_fname_completer(ctxt, cqlsh):
 +    lasttype = ctxt.get_binding('*LASTTYPE*')
 +    if lasttype == 'unclosedString':
 +        return complete_source_quoted_filename(ctxt, cqlsh)
 +    partial_path = ctxt.get_binding('partial')
 +    if partial_path == '':
 +        return ["'"]
 +    return ()
 +
 +
 +@cqlsh_syntax_completer('copyCommand', 'colnames')
 +def complete_copy_column_names(ctxt, cqlsh):
 +    existcols = map(cqlsh.cql_unprotect_name, ctxt.get_binding('colnames', ()))
 +    ks = cqlsh.cql_unprotect_name(ctxt.get_binding('ksname', None))
 +    cf = cqlsh.cql_unprotect_name(ctxt.get_binding('cfname'))
 +    colnames = cqlsh.get_column_names(ks, cf)
 +    if len(existcols) == 0:
 +        return [colnames[0]]
 +    return set(colnames[1:]) - set(existcols)
 +
 +
 +COPY_COMMON_OPTIONS = ['DELIMITER', 'QUOTE', 'ESCAPE', 'HEADER', 'NULL', 'DATETIMEFORMAT',
 +                       'MAXATTEMPTS', 'REPORTFREQUENCY', 'DECIMALSEP', 'THOUSANDSSEP', 'BOOLSTYLE',
 +                       'NUMPROCESSES', 'CONFIGFILE', 'RATEFILE']
 +COPY_FROM_OPTIONS = ['CHUNKSIZE', 'INGESTRATE', 'MAXBATCHSIZE', 'MINBATCHSIZE', 'MAXROWS',
-                      'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE']
++                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE', 'PREPAREDSTATEMENTS']
 +COPY_TO_OPTIONS = ['ENCODING', 'PAGESIZE', 'PAGETIMEOUT', 'BEGINTOKEN', 'ENDTOKEN', 'MAXOUTPUTSIZE', 'MAXREQUESTS']
 +
 +
 +@cqlsh_syntax_completer('copyOption', 'optnames')
 +def complete_copy_options(ctxt, cqlsh):
 +    optnames = map(str.upper, ctxt.get_binding('optnames', ()))
 +    direction = ctxt.get_binding('dir').upper()
 +    if direction == 'FROM':
 +        opts = set(COPY_COMMON_OPTIONS + COPY_FROM_OPTIONS) - set(optnames)
 +    elif direction == 'TO':
 +        opts = set(COPY_COMMON_OPTIONS + COPY_TO_OPTIONS) - set(optnames)
 +    return opts
 +
 +
 +@cqlsh_syntax_completer('copyOption', 'optvals')
 +def complete_copy_opt_values(ctxt, cqlsh):
 +    optnames = ctxt.get_binding('optnames', ())
 +    lastopt = optnames[-1].lower()
 +    if lastopt == 'header':
 +        return ['true', 'false']
 +    return [cqlhandling.Hint('<single_character_string>')]
 +
 +
 +class NoKeyspaceError(Exception):
 +    pass
 +
 +
 +class KeyspaceNotFound(Exception):
 +    pass
 +
 +
 +class ColumnFamilyNotFound(Exception):
 +    pass
 +
 +
 +class IndexNotFound(Exception):
 +    pass
 +
 +
 +class ObjectNotFound(Exception):
 +    pass
 +
 +
 +class VersionNotSupported(Exception):
 +    pass
 +
 +
 +class UserTypeNotFound(Exception):
 +    pass
 +
 +
 +class FunctionNotFound(Exception):
 +    pass
 +
 +
 +class AggregateNotFound(Exception):
 +    pass
 +
 +
 +class DecodeError(Exception):
 +    verb = 'decode'
 +
 +    def __init__(self, thebytes, err, colname=None):
 +        self.thebytes = thebytes
 +        self.err = err
 +        self.colname = colname
 +
 +    def __str__(self):
 +        return str(self.thebytes)
 +
 +    def message(self):
 +        what = 'value %r' % (self.thebytes,)
 +        if self.colname is not None:
 +            what = 'value %r (for column %r)' % (self.thebytes, self.colname)
 +        return 'Failed to %s %s : %s' \
 +               % (self.verb, what, self.err)
 +
 +    def __repr__(self):
 +        return '<%s %s>' % (self.__class__.__name__, self.message())
 +
 +
 +class FormatError(DecodeError):
 +    verb = 'format'
 +
 +
 +def full_cql_version(ver):
 +    while ver.count('.') < 2:
 +        ver += '.0'
 +    ver_parts = ver.split('-', 1) + ['']
 +    vertuple = tuple(map(int, ver_parts[0].split('.')) + [ver_parts[1]])
 +    return ver, vertuple
 +
 +
 +def format_value(val, output_encoding, addcolor=False, date_time_format=None,
 +                 float_precision=None, colormap=None, nullval=None):
 +    if isinstance(val, DecodeError):
 +        if addcolor:
 +            return colorme(repr(val.thebytes), colormap, 'error')
 +        else:
 +            return FormattedValue(repr(val.thebytes))
 +    return format_by_type(type(val), val, output_encoding, colormap=colormap,
 +                          addcolor=addcolor, nullval=nullval, date_time_format=date_time_format,
 +                          float_precision=float_precision)
 +
 +
 +def show_warning_without_quoting_line(message, category, filename, lineno, file=None, line=None):
 +    if file is None:
 +        file = sys.stderr
 +    try:
 +        file.write(warnings.formatwarning(message, category, filename, lineno, line=''))
 +    except IOError:
 +        pass
 +warnings.showwarning = show_warning_without_quoting_line
 +warnings.filterwarnings('always', category=cql3handling.UnexpectedTableStructure)
 +
 +
 +def insert_driver_hooks():
 +    extend_cql_deserialization()
 +    auto_format_udts()
 +
 +
 +def extend_cql_deserialization():
 +    """
-     The python driver returns BLOBs as string, but we expect them as bytearrays
++    The python driver returns BLOBs as string, but we expect them as bytearrays; therefore we change
++    the implementation of cassandra.cqltypes.BytesType.deserialize.
++
++    The deserializers package exists only when the driver has been compiled with cython extensions and
++    cassandra.deserializers.DesBytesType replaces cassandra.cqltypes.BytesType.deserialize.
++
++    DesBytesTypeByteArray is a fast deserializer that converts blobs into bytearrays but it was
++    only introduced recently (3.1.0). If it is available we use it, otherwise we remove
++    cassandra.deserializers.DesBytesType so that we fall back onto cassandra.cqltypes.BytesType.deserialize
++    just like in the case where no cython extensions are present.
 +    """
++    if hasattr(cassandra, 'deserializers'):
++        if hasattr(cassandra.deserializers, 'DesBytesTypeByteArray'):
++            cassandra.deserializers.DesBytesType = cassandra.deserializers.DesBytesTypeByteArray
++        else:
++            del cassandra.deserializers.DesBytesType
++
 +    cassandra.cqltypes.BytesType.deserialize = staticmethod(lambda byts, protocol_version: bytearray(byts))
 +    cassandra.cqltypes.CassandraType.support_empty_values = True
 +
 +
 +def auto_format_udts():
 +    # when we see a new user defined type, set up the shell formatting for it
 +    udt_apply_params = cassandra.cqltypes.UserType.apply_parameters
 +
 +    def new_apply_params(cls, *args, **kwargs):
 +        udt_class = udt_apply_params(*args, **kwargs)
 +        formatter_for(udt_class.typename)(format_value_utype)
 +        return udt_class
 +
 +    cassandra.cqltypes.UserType.udt_apply_parameters = classmethod(new_apply_params)
 +
 +    make_udt_class = cassandra.cqltypes.UserType.make_udt_class
 +
 +    def new_make_udt_class(cls, *args, **kwargs):
 +        udt_class = make_udt_class(*args, **kwargs)
 +        formatter_for(udt_class.tuple_type.__name__)(format_value_utype)
 +        return udt_class
 +
 +    cassandra.cqltypes.UserType.make_udt_class = classmethod(new_make_udt_class)
 +
 +
 +class FrozenType(cassandra.cqltypes._ParameterizedType):
 +    """
 +    Needed until the bundled python driver adds FrozenType.
 +    """
 +    typename = "frozen"
 +    num_subtypes = 1
 +
 +    @classmethod
 +    def deserialize_safe(cls, byts, protocol_version):
 +        subtype, = cls.subtypes
 +        return subtype.from_binary(byts)
 +
 +    @classmethod
 +    def serialize_safe(cls, val, protocol_version):
 +        subtype, = cls.subtypes
 +        return subtype.to_binary(val, protocol_version)
 +
 +
 +class Shell(cmd.Cmd):
 +    custom_prompt = os.getenv('CQLSH_PROMPT', '')
 +    if custom_prompt is not '':
 +        custom_prompt += "\n"
 +    default_prompt = custom_prompt + "cqlsh> "
 +    continue_prompt = "   ... "
 +    keyspace_prompt = custom_prompt + "cqlsh:%s> "
 +    keyspace_continue_prompt = "%s    ... "
 +    show_line_nums = False
 +    debug = False
 +    stop = False
 +    last_hist = None
 +    shunted_query_out = None
 +    use_paging = True
 +
 +    default_page_size = 100
 +
 +    def __init__(self, hostname, port, color=False,
 +                 username=None, password=None, encoding=None, stdin=None, tty=True,
 +                 completekey=DEFAULT_COMPLETEKEY, browser=None, use_conn=None,
 +                 cqlver=DEFAULT_CQLVER, keyspace=None,
 +                 tracing_enabled=False, expand_enabled=False,
 +                 display_nanotime_format=DEFAULT_NANOTIME_FORMAT,
 +                 display_timestamp_format=DEFAULT_TIMESTAMP_FORMAT,
 +                 display_date_format=DEFAULT_DATE_FORMAT,
 +                 display_float_precision=DEFAULT_FLOAT_PRECISION,
 +                 display_timezone=None,
 +                 max_trace_wait=DEFAULT_MAX_TRACE_WAIT,
 +                 ssl=False,
 +                 single_statement=None,
 +                 request_timeout=DEFAULT_REQUEST_TIMEOUT_SECONDS,
 +                 protocol_version=DEFAULT_PROTOCOL_VERSION,
 +                 connect_timeout=DEFAULT_CONNECT_TIMEOUT_SECONDS):
 +        cmd.Cmd.__init__(self, completekey=completekey)
 +        self.hostname = hostname
 +        self.port = port
 +        self.auth_provider = None
 +        if username:
 +            if not password:
 +                password = getpass.getpass()
 +            self.auth_provider = PlainTextAuthProvider(username=username, password=password)
 +        self.username = username
 +        self.keyspace = keyspace
 +        self.ssl = ssl
 +        self.tracing_enabled = tracing_enabled
 +        self.page_size = self.default_page_size
 +        self.expand_enabled = expand_enabled
 +        if use_conn:
 +            self.conn = use_conn
 +        else:
 +            self.conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=cqlver,
 +                                protocol_version=protocol_version,
 +                                auth_provider=self.auth_provider,
 +                                ssl_options=sslhandling.ssl_settings(hostname, CONFIG_FILE) if ssl else None,
 +                                load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]),
 +                                control_connection_timeout=connect_timeout,
 +                                connect_timeout=connect_timeout)
 +        self.owns_connection = not use_conn
 +        self.set_expanded_cql_version(cqlver)
 +
 +        if keyspace:
 +            self.session = self.conn.connect(keyspace)
 +        else:
 +            self.session = self.conn.connect()
 +
 +        if browser == "":
 +            browser = None
 +        self.browser = browser
 +        self.color = color
 +
 +        self.display_nanotime_format = display_nanotime_format
 +        self.display_timestamp_format = display_timestamp_format
 +        self.display_date_format = display_date_format
 +
 +        self.display_float_precision = display_float_precision
 +
 +        self.display_timezone = display_timezone
 +
 +        # If there is no schema metadata present (due to a schema mismatch), force schema refresh
 +        if not self.conn.metadata.keyspaces:
 +            self.refresh_schema_metadata_best_effort()
 +
 +        self.session.default_timeout = request_timeout
 +        self.session.row_factory = ordered_dict_factory
 +        self.session.default_consistency_level = cassandra.ConsistencyLevel.ONE
 +        self.get_connection_versions()
 +
 +        self.current_keyspace = keyspace
 +
 +        self.display_timestamp_format = display_timestamp_format
 +        self.display_nanotime_format = display_nanotime_format
 +        self.display_date_format = display_date_format
 +
 +        self.max_trace_wait = max_trace_wait
 +        self.session.max_trace_wait = max_trace_wait
 +
 +        self.tty = tty
 +        self.encoding = encoding
 +        self.check_windows_encoding()
 +
 +        self.output_codec = codecs.lookup(encoding)
 +
 +        self.statement = StringIO()
 +        self.lineno = 1
 +        self.in_comment = False
 +
 +        self.prompt = ''
 +        if stdin is None:
 +            stdin = sys.stdin
 +
 +        if tty:
 +            self.reset_prompt()
 +            self.report_connection()
 +            print 'Use HELP for help.'
 +        else:
 +            self.show_line_nums = True
 +        self.stdin = stdin
 +        self.query_out = sys.stdout
 +        self.consistency_level = cassandra.ConsistencyLevel.ONE
 +        self.serial_consistency_level = cassandra.ConsistencyLevel.SERIAL
 +
 +        self.empty_lines = 0
 +        self.statement_error = False
 +        self.single_statement = single_statement
 +
 +    @property
 +    def is_using_utf8(self):
 +        # utf8 encodings from https://docs.python.org/{2,3}/library/codecs.html
 +        return self.encoding.replace('-', '_').lower() in ['utf', 'utf_8', 'u8', 'utf8', CP65001]
 +
 +    def check_windows_encoding(self):
 +        if is_win and os.name == 'nt' and self.tty and \
 +           self.is_using_utf8 and sys.stdout.encoding != CP65001:
 +            self.printerr("\nWARNING: console codepage must be set to cp65001 "
 +                          "to support {} encoding on Windows platforms.\n"
 +                          "If you experience encoding problems, change your console"
 +                          " codepage with 'chcp 65001' before starting cqlsh.\n".format(self.encoding))
 +
 +    def refresh_schema_metadata_best_effort(self):
 +        try:
 +            self.conn.refresh_schema_metadata(5)  # will throw exception if there is a schema mismatch
 +        except Exception:
 +            self.printerr("Warning: schema version mismatch detected, which might be caused by DOWN nodes; if "
 +                          "this is not the case, check the schema versions of your nodes in system.local and "
 +                          "system.peers.")
 +            self.conn.refresh_schema_metadata(0)
 +
 +    def set_expanded_cql_version(self, ver):
 +        ver, vertuple = full_cql_version(ver)
 +        self.cql_version = ver
 +        self.cql_ver_tuple = vertuple
 +
 +    def cqlver_atleast(self, major, minor=0, patch=0):
 +        return self.cql_ver_tuple[:3] >= (major, minor, patch)
 +
 +    def myformat_value(self, val, **kwargs):
 +        if isinstance(val, DecodeError):
 +            self.decoding_errors.append(val)
 +        try:
 +            dtformats = DateTimeFormat(timestamp_format=self.display_timestamp_format,
 +                                       date_format=self.display_date_format, nanotime_format=self.display_nanotime_format,
 +                                       timezone=self.display_timezone)
 +            return format_value(val, self.output_codec.name,
 +                                addcolor=self.color, date_time_format=dtformats,
 +                                float_precision=self.display_float_precision, **kwargs)
 +        except Exception, e:
 +            err = FormatError(val, e)
 +            self.decoding_errors.append(err)
 +            return format_value(err, self.output_codec.name, addcolor=self.color)
 +
 +    def myformat_colname(self, name, table_meta=None):
 +        column_colors = COLUMN_NAME_COLORS.copy()
 +        # check column role and color appropriately
 +        if table_meta:
 +            if name in [col.name for col in table_meta.partition_key]:
 +                column_colors.default_factory = lambda: RED
 +            elif name in [col.name for col in table_meta.clustering_key]:
 +                column_colors.default_factory = lambda: CYAN
 +        return self.myformat_value(name, colormap=column_colors)
 +
 +    def report_connection(self):
 +        self.show_host()
 +        self.show_version()
 +
 +    def show_host(self):
 +        print "Connected to %s at %s:%d." % \
 +            (self.applycolor(self.get_cluster_name(), BLUE),
 +              self.hostname,
 +              self.port)
 +
 +    def show_version(self):
 +        vers = self.connection_versions.copy()
 +        vers['shver'] = version
 +        # system.Versions['cql'] apparently does not reflect changes with
 +        # set_cql_version.
 +        vers['cql'] = self.cql_version
 +        print "[cqlsh %(shver)s | Cassandra %(build)s | CQL spec %(cql)s | Native protocol v%(protocol)s]" % vers
 +
 +    def show_session(self, sessionid, partial_session=False):
 +        print_trace_session(self, self.session, sessionid, partial_session)
 +
 +    def get_connection_versions(self):
 +        result, = self.session.execute("select * from system.local where key = 'local'")
 +        vers = {
 +            'build': result['release_version'],
 +            'protocol': result['native_protocol_version'],
 +            'cql': result['cql_version'],
 +        }
 +        self.connection_versions = vers
 +
 +    def get_keyspace_names(self):
 +        return map(str, self.conn.metadata.keyspaces.keys())
 +
 +    def get_columnfamily_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return map(str, self.get_keyspace_meta(ksname).tables.keys())
 +
 +    def get_index_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return map(str, self.get_keyspace_meta(ksname).indexes.keys())
 +
 +    def get_column_names(self, ksname, cfname):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        layout = self.get_table_meta(ksname, cfname)
 +        return [str(col) for col in layout.columns]
 +
 +    def get_usertype_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return self.get_keyspace_meta(ksname).user_types.keys()
 +
 +    def get_usertype_layout(self, ksname, typename):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        ks_meta = self.get_keyspace_meta(ksname)
 +
 +        try:
 +            user_type = ks_meta.user_types[typename]
 +        except KeyError:
 +            raise UserTypeNotFound("User type %r not found" % typename)
 +
 +        return [(field_name, field_type.cql_parameterized_type())
 +                for field_name, field_type in zip(user_type.field_names, user_type.field_types)]
 +
 +    def get_userfunction_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return map(lambda f: f.name, self.get_keyspace_meta(ksname).functions.values())
 +
 +    def get_useraggregate_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return map(lambda f: f.name, self.get_keyspace_meta(ksname).aggregates.values())
 +
 +    def get_cluster_name(self):
 +        return self.conn.metadata.cluster_name
 +
 +    def get_partitioner(self):
 +        return self.conn.metadata.partitioner
 +
 +    def get_keyspace_meta(self, ksname):
 +        if ksname not in self.conn.metadata.keyspaces:
 +            raise KeyspaceNotFound('Keyspace %r not found.' % ksname)
 +        return self.conn.metadata.keyspaces[ksname]
 +
 +    def get_keyspaces(self):
 +        return self.conn.metadata.keyspaces.values()
 +
 +    def get_ring(self, ks):
 +        self.conn.metadata.token_map.rebuild_keyspace(ks, build_if_absent=True)
 +        return self.conn.metadata.token_map.tokens_to_hosts_by_ks[ks]
 +
 +    def get_table_meta(self, ksname, tablename):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        ksmeta = self.get_keyspace_meta(ksname)
 +
 +        if tablename not in ksmeta.tables:
 +            if ksname == 'system_auth' and tablename in ['roles', 'role_permissions']:
 +                self.get_fake_auth_table_meta(ksname, tablename)
 +            else:
 +                raise ColumnFamilyNotFound("Column family %r not found" % tablename)
 +        else:
 +            return ksmeta.tables[tablename]
 +
 +    def get_fake_auth_table_meta(self, ksname, tablename):
 +        # may be using external auth implementation so internal tables
 +        # aren't actually defined in schema. In this case, we'll fake
 +        # them up
 +        if tablename == 'roles':
 +            ks_meta = KeyspaceMetadata(ksname, True, None, None)
 +            table_meta = TableMetadata(ks_meta, 'roles')
 +            table_meta.columns['role'] = ColumnMetadata(table_meta, 'role', cassandra.cqltypes.UTF8Type)
 +            table_meta.columns['is_superuser'] = ColumnMetadata(table_meta, 'is_superuser', cassandra.cqltypes.BooleanType)
 +            table_meta.columns['can_login'] = ColumnMetadata(table_meta, 'can_login', cassandra.cqltypes.BooleanType)
 +        elif tablename == 'role_permissions':
 +            ks_meta = KeyspaceMetadata(ksname, True, None, None)
 +            table_meta = TableMetadata(ks_meta, 'role_permissions')
 +            table_meta.columns['role'] = ColumnMetadata(table_meta, 'role', cassandra.cqltypes.UTF8Type)
 +            table_meta.columns['resource'] = ColumnMetadata(table_meta, 'resource', cassandra.cqltypes.UTF8Type)
 +            table_meta.columns['permission'] = ColumnMetadata(table_meta, 'permission', cassandra.cqltypes.UTF8Type)
 +        else:
 +            raise ColumnFamilyNotFound("Column family %r not found" % tablename)
 +
 +    def get_index_meta(self, ksname, idxname):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        ksmeta = self.get_keyspace_meta(ksname)
 +
 +        if idxname not in ksmeta.indexes:
 +            raise IndexNotFound("Index %r not found" % idxname)
 +
 +        return ksmeta.indexes[idxname]
 +
 +    def get_object_meta(self, ks, name):
 +        if name is None:
 +            if ks and ks in self.conn.metadata.keyspaces:
 +                return self.conn.metadata.keyspaces[ks]
 +            elif self.current_keyspace is None:
 +                raise ObjectNotFound("%r not found in keyspaces" % (ks))
 +            else:
 +                name = ks
 +                ks = self.current_keyspace
 +
 +        if ks is None:
 +            ks = self.current_keyspace
 +
 +        ksmeta = self.get_keyspace_meta(ks)
 +
 +        if name in ksmeta.tables:
 +            return ksmeta.tables[name]
 +        elif name in ksmeta.indexes:
 +            return ksmeta.indexes[name]
 +
 +        raise ObjectNotFound("%r not found in keyspace %r" % (name, ks))
 +
 +    def get_usertypes_meta(self):
 +        data = self.session.execute("select * from system.schema_usertypes")
 +        if not data:
 +            return cql3handling.UserTypesMeta({})
 +
 +        return cql3handling.UserTypesMeta.from_layout(data)
 +
 +    def get_trigger_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return [trigger.name
 +                for table in self.get_keyspace_meta(ksname).tables.values()
 +                for trigger in table.triggers.values()]
 +
 +    def reset_statement(self):
 +        self.reset_prompt()
 +        self.statement.truncate(0)
 +        self.empty_lines = 0
 +
 +    def reset_prompt(self):
 +        if self.current_keyspace is None:
 +            self.set_prompt(self.default_prompt, True)
 +        else:
 +            self.set_prompt(self.keyspace_prompt % self.current_keyspace, True)
 +
 +    def set_continue_prompt(self):
 +        if self.empty_lines >= 3:
 +            self.set_prompt("Statements are terminated with a ';'.  You can press CTRL-C to cancel an incomplete statement.")
 +            self.empty_lines = 0
 +            return
 +        if self.current_keyspace is None:
 +            self.set_prompt(self.continue_prompt)
 +        else:
 +            spaces = ' ' * len(str(self.current_keyspace))
 +            self.set_prompt(self.keyspace_continue_prompt % spaces)
 +        self.empty_lines = self.empty_lines + 1 if not self.lastcmd else 0
 +
 +    @contextmanager
 +    def prepare_loop(self):
 +        readline = None
 +        if self.tty and self.completekey:
 +            try:
 +                import readline
 +            except ImportError:
 +                if is_win:
 +                    print "WARNING: pyreadline dependency missing.  Install to enable tab completion."
 +                pass
 +            else:
 +                old_completer = readline.get_completer()
 +                readline.set_completer(self.complete)
 +                if readline.__doc__ is not None and 'libedit' in readline.__doc__:
 +                    readline.parse_and_bind("bind -e")
 +                    readline.parse_and_bind("bind '" + self.completekey + "' rl_complete")
 +                    readline.parse_and_bind("bind ^R em-inc-search-prev")
 +                else:
 +                    readline.parse_and_bind(self.completekey + ": complete")
 +        try:
 +            yield
 +        finally:
 +            if readline is not None:
 +                readline.set_completer(old_completer)
 +
 +    def get_input_line(self, prompt=''):
 +        if self.tty:
 +            try:
 +                self.lastcmd = raw_input(prompt).decode(self.encoding)
 +            except UnicodeDecodeError:
 +                self.lastcmd = ''
 +                traceback.print_exc()
 +                self.check_windows_encoding()
 +            line = self.lastcmd + '\n'
 +        else:
 +            self.lastcmd = self.stdin.readline()
 +            line = self.lastcmd
 +            if not len(line):
 +                raise EOFError
 +        self.lineno += 1
 +        return line
 +
 +    def use_stdin_reader(self, until='', prompt=''):
 +        until += '\n'
 +        while True:
 +            try:
 +                newline = self.get_input_line(prompt=prompt)
 +            except EOFError:
 +                return
 +            if newline == until:
 +                return
 +            yield newline
 +
 +    def cmdloop(self):
 +        """
 +        Adapted from cmd.Cmd's version, because there is literally no way with
 +        cmd.Cmd.cmdloop() to tell the difference between "EOF" showing up in
 +        input and an actual EOF.
 +        """
 +        with self.prepare_loop():
 +            while not self.stop:
 +                try:
 +                    if self.single_statement:
 +                        line = self.single_statement
 +                        self.stop = True
 +                    else:
 +                        line = self.get_input_line(self.prompt)
 +                    self.statement.write(line)
 +                    if self.onecmd(self.statement.getvalue()):
 +                        self.reset_statement()
 +                except EOFError:
 +                    self.handle_eof()
 +                except CQL_ERRORS, cqlerr:
 +                    self.printerr(str(cqlerr))
 +                except KeyboardInterrupt:
 +                    self.reset_statement()
 +                    print
 +
 +    def onecmd(self, statementtext):
 +        """
 +        Returns true if the statement is complete and was handled (meaning it
 +        can be reset).
 +        """
 +
 +        try:
 +            statements, endtoken_escaped = cqlruleset.cql_split_statements(statementtext)
 +        except pylexotron.LexingError, e:
 +            if self.show_line_nums:
 +                self.printerr('Invalid syntax at char %d' % (e.charnum,))
 +            else:
 +                self.printerr('Invalid syntax at line %d, char %d'
 +                              % (e.linenum, e.charnum))
 +            statementline = statementtext.split('\n')[e.linenum - 1]
 +            self.printerr('  %s' % statementline)
 +            self.printerr(' %s^' % (' ' * e.charnum))
 +            return True
 +
 +        while statements and not statements[-1]:
 +            statements = statements[:-1]
 +        if not statements:
 +            return True
 +        if endtoken_escaped or statements[-1][-1][0] != 'endtoken':
 +            self.set_continue_prompt()
 +            return
 +        for st in statements:
 +            try:
 +                self.handle_statement(st, statementtext)
 +            except Exception, e:
 +                if self.debug:
 +                    traceback.print_exc()
 +                else:
 +                    self.printerr(e)
 +        return True
 +
 +    def handle_eof(self):
 +        if self.tty:
 +            print
 +        statement = self.statement.getvalue()
 +        if statement.strip():
 +            if not self.onecmd(statement):
 +                self.printerr('Incomplete statement at end of file')
 +        self.do_exit()
 +
 +    def handle_statement(self, tokens, srcstr):
 +        # Concat multi-line statements and insert into history
 +        if readline is not None:
 +            nl_count = srcstr.count("\n")
 +
 +            new_hist = srcstr.replace("\n", " ").rstrip()
 +
 +            if nl_count > 1 and self.last_hist != new_hist:
 +                readline.add_history(new_hist.encode(self.encoding))
 +
 +            self.last_hist = new_hist
 +        cmdword = tokens[0][1]
 +        if cmdword == '?':
 +            cmdword = 'help'
 +        custom_handler = getattr(self, 'do_' + cmdword.lower(), None)
 +        if custom_handler:
 +            parsed = cqlruleset.cql_whole_parse_tokens(tokens, srcstr=srcstr,
 +                                                       startsymbol='cqlshCommand')
 +            if parsed and not parsed.remainder:
 +                # successful complete parse
 +                return custom_handler(parsed)
 +            else:
 +                return self.handle_parse_error(cmdword, tokens, parsed, srcstr)
 +        return self.perform_statement(cqlruleset.cql_extract_orig(tokens, srcstr))
 +
 +    def handle_parse_error(self, cmdword, tokens, parsed, srcstr):
 +        if cmdword.lower() in ('select', 'insert', 'update', 'delete', 'truncate',
 +                               'create', 'drop', 'alter', 'grant', 'revoke',
 +                               'batch', 'list'):
 +            # hey, maybe they know about some new syntax we don't. type
 +            # assumptions won't work, but maybe the query will.
 +            return self.perform_statement(cqlruleset.cql_extract_orig(tokens, srcstr))
 +        if parsed:
 +            self.printerr('Improper %s command (problem at %r).' % (cmdword, parsed.remainder[0]))
 +        else:
 +            self.printerr('Improper %s command.' % cmdword)
 +
 +    def do_use(self, parsed):
 +        ksname = parsed.get_binding('ksname')
 +        success, _ = self.perform_simple_statement(SimpleStatement(parsed.extract_orig()))
 +        if success:
 +            if ksname[0] == '"' and ksname[-1] == '"':
 +                self.current_keyspace = self.cql_unprotect_name(ksname)
 +            else:
 +                self.current_keyspace = ksname.lower()
 +
 +    def do_select(self, parsed):
 +        tracing_was_enabled = self.tracing_enabled
 +        ksname = parsed.get_binding('ksname')
 +        stop_tracing = ksname == 'system_traces' or (ksname is None and self.current_keyspace == 'system_traces')
 +        self.tracing_enabled = self.tracing_enabled and not stop_tracing
 +        statement = parsed.extract_orig()
 +        self.perform_statement(statement)
 +        self.tracing_enabled = tracing_was_enabled
 +
 +    def perform_statement(self, statement):
 +        stmt = SimpleStatement(statement, consistency_level=self.consistency_level, serial_consistency_level=self.serial_consistency_level, fetch_size=self.page_size if self.use_paging else None)
 +        success, future = self.perform_simple_statement(stmt)
 +
 +        if future:
 +            if future.warnings:
 +                self.print_warnings(future.warnings)
 +
 +            if self.tracing_enabled:
 +                try:
 +                    for trace in future.get_all_query_traces(self.max_trace_wait):
 +                        print_trace(self, trace)
 +                except TraceUnavailable:
 +                    msg = "Statement trace did not complete within %d seconds; trace data may be incomplete." % (self.session.max_trace_wait,)
 +                    self.writeresult(msg, color=RED)
 +                    for trace_id in future.get_query_trace_ids():
 +                        self.show_session(trace_id, partial_session=True)
 +                except Exception, err:
 +                    self.printerr("Unable to fetch query trace: %s" % (str(err),))
 +
 +        return success
 +
 +    def parse_for_table_meta(self, query_string):
 +        try:
 +            parsed = cqlruleset.cql_parse(query_string)[1]
 +        except IndexError:
 +            return None
 +        ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +        cf = self.cql_unprotect_name(parsed.get_binding('cfname'))
 +        return self.get_table_meta(ks, cf)
 +
 +    def perform_simple_statement(self, statement):
 +        if not statement:
 +            return False, None
 +
 +        while True:
 +            try:
 +                future = self.session.execute_async(statement, trace=self.tracing_enabled)
 +                result = future.result()
 +                break
 +            except cassandra.OperationTimedOut, err:
 +                self.refresh_schema_metadata_best_effort()
 +                self.printerr(str(err.__class__.__name__) + ": " + str(err))
 +                return False, None
 +            except CQL_ERRORS, err:
 +                self.printerr(str(err.__class__.__name__) + ": " + str(err))
 +                return False, None
 +            except Exception, err:
 +                import traceback
 +                self.printerr(traceback.format_exc())
 +                return False, None
 +
 +        if statement.query_string[:6].lower() == 'select':
 +            self.print_result(result, self.parse_for_table_meta(statement.query_string))
 +        elif statement.query_string.lower().startswith("list users") or statement.query_string.lower().startswith("list roles"):
 +            self.print_result(result, self.get_table_meta('system_auth', 'roles'))
 +        elif statement.query_string.lower().startswith("list"):
 +            self.print_result(result, self.get_table_meta('system_auth', 'role_permissions'))
 +        elif result:
 +            # CAS INSERT/UPDATE
 +            self.writeresult("")
 +            self.print_static_result(result.column_names, list(result), self.parse_for_table_meta(statement.query_string))
 +        self.flush_output()
 +        return True, future
 +
 +    def print_result(self, result, table_meta):
 +        self.decoding_errors = []
 +
 +        self.writeresult("")
 +        if result.has_more_pages and self.tty:
 +            num_rows = 0
 +            while True:
 +                page = result.current_rows
 +                if page:
 +                    num_rows += len(page)
 +                    self.print_static_result(result.column_names, page, table_meta)
 +                if result.has_more_pages:
 +                    raw_input("---MORE---")
 +                    result.fetch_next_page()
 +                else:
 +                    break
 +        else:
 +            rows = list(result)
 +            num_rows = len(rows)
 +            self.print_static_result(result.column_names, rows, table_meta)
 +        self.writeresult("(%d rows)" % num_rows)
 +
 +        if self.decoding_errors:
 +            for err in self.decoding_errors[:2]:
 +                self.writeresult(err.message(), color=RED)
 +            if len(self.decoding_errors) > 2:
 +                self.writeresult('%d more decoding errors suppressed.'
 +                                 % (len(self.decoding_errors) - 2), color=RED)
 +
 +    def print_static_result(self, column_names, rows, table_meta):
 +        if not column_names and not table_meta:
 +            return
 +        column_names = column_names or table_meta.columns.keys()
 +        formatted_names = [self.myformat_colname(name, table_meta) for name in column_names]
 +        formatted_values = [map(self.myformat_value, row.values()) for row in rows]
 +
 +        if self.expand_enabled:
 +            self.print_formatted_result_vertically(formatted_names, formatted_values)
 +        else:
 +            self.print_formatted_result(formatted_names, formatted_values)
 +
 +    def print_formatted_result(self, formatted_names, formatted_values):
 +        # determine column widths
 +        widths = [n.displaywidth for n in formatted_names]
 +        if formatted_values is not None:
 +            for fmtrow in formatted_values:
 +                for num, col in enumerate(fmtrow):
 +                    widths[num] = max(widths[num], col.displaywidth)
 +
 +        # print header
 +        header = ' | '.join(hdr.ljust(w, color=self.color) for (hdr, w) in zip(formatted_names, widths))
 +        self.writeresult(' ' + header.rstrip())
 +        self.writeresult('-%s-' % '-+-'.join('-' * w for w in widths))
 +
 +        # stop if there are no rows
 +        if formatted_values is None:
 +            self.writeresult("")
 +            return
 +
 +        # print row data
 +        for row in formatted_values:
 +            line = ' | '.join(col.rjust(w, color=self.color) for (col, w) in zip(row, widths))
 +            self.writeresult(' ' + line)
 +
 +        self.writeresult("")
 +
 +    def print_formatted_result_vertically(self, formatted_names, formatted_values):
 +        max_col_width = max([n.displaywidth for n in formatted_names])
 +        max_val_width = max([n.displaywidth for row in formatted_values for n in row])
 +
 +        # for each row returned, list all the column-value pairs
 +        for row_id, row in enumerate(formatted_values):
 +            self.writeresult("@ Row %d" % (row_id + 1))
 +            self.writeresult('-%s-' % '-+-'.join(['-' * max_col_width, '-' * max_val_width]))
 +            for field_id, field in enumerate(row):
 +                column = formatted_names[field_id].ljust(max_col_width, color=self.color)
 +                value = field.ljust(field.displaywidth, color=self.color)
 +                self.writeresult(' ' + " | ".join([column, value]))
 +            self.writeresult('')
 +
 +    def print_warnings(self, warnings):
 +        if warnings is None or len(warnings) == 0:
 +            return
 +
 +        self.writeresult('')
 +        self.writeresult('Warnings :')
 +        for warning in warnings:
 +            self.writeresult(warning)
 +            self.writeresult('')
 +
 +    def emptyline(self):
 +        pass
 +
 +    def parseline(self, line):
 +        # this shouldn't be needed
 +        raise NotImplementedError
 +
 +    def complete(self, text, state):
 +        if readline is None:
 +            return
 +        if state == 0:
 +            try:
 +                self.completion_matches = self.find_completions(text)
 +            except Exception:
 +                if debug_completion:
 +                    import traceback
 +                    traceback.print_exc()
 +                else:
 +                    raise
 +        try:
 +            return self.completion_matches[state]
 +        except IndexError:
 +            return None
 +
 +    def find_completions(self, text):
 +        curline = readline.get_line_buffer()
 +        prevlines = self.statement.getvalue()
 +        wholestmt = prevlines + curline
 +        begidx = readline.get_begidx() + len(prevlines)
 +        stuff_to_complete = wholestmt[:begidx]
 +        return cqlruleset.cql_complete(stuff_to_complete, text, cassandra_conn=self,
 +                                       debug=debug_completion, startsymbol='cqlshCommand')
 +
 +    def set_prompt(self, prompt, prepend_user=False):
 +        if prepend_user and self.username:
 +            self.prompt = "%s@%s" % (self.username, prompt)
 +            return
 +        self.prompt = prompt
 +
 +    def cql_unprotect_name(self, namestr):
 +        if namestr is None:
 +            return
 +        return cqlruleset.dequote_name(namestr)
 +
 +    def cql_unprotect_value(self, valstr):
 +        if valstr is not None:
 +            return cqlruleset.dequote_value(valstr)
 +
 +    def print_recreate_keyspace(self, ksdef, out):
 +        out.write(ksdef.export_as_string())
 +        out.write("\n")
 +
 +    def print_recreate_columnfamily(self, ksname, cfname, out):
 +        """
 +        Output CQL commands which should be pasteable back into a CQL session
 +        to recreate the given table.
 +
 +        Writes output to the given out stream.
 +        """
 +        out.write(self.get_table_meta(ksname, cfname).export_as_string())
 +        out.write("\n")
 +
 +    def print_recreate_index(self, ksname, idxname, out):
 +        """
 +        Output CQL commands which should be pasteable back into a CQL session
 +        to recreate the given index.
 +
 +        Writes output to the given out stream.
 +        """
 +        out.write(self.get_index_meta(ksname, idxname).export_as_string())
 +        out.write("\n")
 +
 +    def print_recreate_object(self, ks, name, out):
 +        """
 +        Output CQL commands which should be pasteable back into a CQL session
 +        to recreate the given object (ks, table or index).
 +
 +        Writes output to the given out stream.
 +        """
 +        out.write(self.get_object_meta(ks, name).export_as_string())
 +        out.write("\n")
 +
 +    def describe_keyspaces(self):
 +        print
 +        cmd.Cmd.columnize(self, protect_names(self.get_keyspace_names()))
 +        print
 +
 +    def describe_keyspace(self, ksname):
 +        print
 +        self.print_recreate_keyspace(self.get_keyspace_meta(ksname), sys.stdout)
 +        print
 +
 +    def describe_columnfamily(self, ksname, cfname):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        if ksname is None:
 +            raise NoKeyspaceError("No keyspace specified and no current keyspace")
 +        print
 +        self.print_recreate_columnfamily(ksname, cfname, sys.stdout)
 +        print
 +
 +    def describe_index(self, ksname, idxname):
 +        print
 +        self.print_recreate_index(ksname, idxname, sys.stdout)
 +        print
 +
 +    def describe_object(self, ks, name):
 +        print
 +        self.print_recreate_object(ks, name, sys.stdout)
 +        print
 +
 +    def describe_columnfamilies(self, ksname):
 +        print
 +        if ksname is None:
 +            for k in self.get_keyspaces():
 +                name = protect_name(k.name)
 +                print 'Keyspace %s' % (name,)
 +                print '---------%s' % ('-' * len(name))
 +                cmd.Cmd.columnize(self, protect_names(self.get_columnfamily_names(k.name)))
 +                print
 +        else:
 +            cmd.Cmd.columnize(self, protect_names(self.get_columnfamily_names(ksname)))
 +            print
 +
 +    def describe_functions(self, ksname):
 +        print
 +        if ksname is None:
 +            for ksmeta in self.get_keyspaces():
 +                name = protect_name(ksmeta.name)
 +                print 'Keyspace %s' % (name,)
 +                print '---------%s' % ('-' * len(name))
 +                cmd.Cmd.columnize(self, protect_names(ksmeta.functions.keys()))
 +                print
 +        else:
 +            ksmeta = self.get_keyspace_meta(ksname)
 +            cmd.Cmd.columnize(self, protect_names(ksmeta.functions.keys()))
 +            print
 +
 +    def describe_function(self, ksname, functionname):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        if ksname is None:
 +            raise NoKeyspaceError("No keyspace specified and no current keyspace")
 +        print
 +        ksmeta = self.get_keyspace_meta(ksname)
 +        functions = filter(lambda f: f.name == functionname, ksmeta.functions.values())
 +        if len(functions) == 0:
 +            raise FunctionNotFound("User defined function %r not found" % functionname)
 +        print "\n\n".join(func.as_cql_query(formatted=True) for func in functions)
 +        print
 +
 +    def describe_aggregates(self, ksname):
 +        print
 +        if ksname is None:
 +            for ksmeta in self.get_keyspaces():
 +                name = protect_name(ksmeta.name)
 +                print 'Keyspace %s' % (name,)
 +                print '---------%s' % ('-' * len(name))
 +                cmd.Cmd.columnize(self, protect_names(ksmeta.aggregates.keys()))
 +                print
 +        else:
 +            ksmeta = self.get_keyspace_meta(ksname)
 +            cmd.Cmd.columnize(self, protect_names(ksmeta.aggregates.keys()))
 +            print
 +
 +    def describe_aggregate(self, ksname, aggregatename):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        if ksname is None:
 +            raise NoKeyspaceError("No keyspace specified and no current keyspace")
 +        print
 +        ksmeta = self.get_keyspace_meta(ksname)
 +        aggregates = filter(lambda f: f.name == aggregatename, ksmeta.aggregates.values())
 +        if len(aggregates) == 0:
 +            raise FunctionNotFound("User defined aggregate %r not found" % aggregatename)
 +        print "\n\n".join(aggr.as_cql_query(formatted=True) for aggr in aggregates)
 +        print
 +
 +    def describe_usertypes(self, ksname):
 +        print
 +        if ksname is None:
 +            for ksmeta in self.get_keyspaces():
 +                name = protect_name(ksmeta.name)
 +                print 'Keyspace %s' % (name,)
 +                print '---------%s' % ('-' * len(name))
 +                cmd.Cmd.columnize(self, protect_names(ksmeta.user_types.keys()))
 +                print
 +        else:
 +            ksmeta = self.get_keyspace_meta(ksname)
 +            cmd.Cmd.columnize(self, protect_names(ksmeta.user_types.keys()))
 +            print
 +
 +    def describe_usertype(self, ksname, typename):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        if ksname is None:
 +            raise NoKeyspaceError("No keyspace specified and no current keyspace")
 +        print
 +        ksmeta = self.get_keyspace_meta(ksname)
 +        try:
 +            usertype = ksmeta.user_types[typename]
 +        except KeyError:
 +            raise UserTypeNotFound("User type %r not found" % typename)
 +        print usertype.as_cql_query(formatted=True)
 +        print
 +
 +    def describe_cluster(self):
 +        print '\nCluster: %s' % self.get_cluster_name()
 +        p = trim_if_present(self.get_partitioner(), 'org.apache.cassandra.dht.')
 +        print 'Partitioner: %s\n' % p
 +        # TODO: snitch?
 +        # snitch = trim_if_present(self.get_snitch(), 'org.apache.cassandra.locator.')
 +        # print 'Snitch: %s\n' % snitch
 +        if self.current_keyspace is not None and self.current_keyspace != 'system':
 +            print "Range ownership:"
 +            ring = self.get_ring(self.current_keyspace)
 +            for entry in ring.items():
 +                print ' %39s  [%s]' % (str(entry[0].value), ', '.join([host.address for host in entry[1]]))
 +            print
 +
 +    def describe_schema(self, include_system=False):
 +        print
 +        for k in self.get_keyspaces():
 +            if include_system or k.name not in cql3handling.SYSTEM_KEYSPACES:
 +                self.print_recreate_keyspace(k, sys.stdout)
 +                print
 +
 +    def do_describe(self, parsed):
 +        """
 +        DESCRIBE [cqlsh only]
 +
 +        (DESC may be used as a shorthand.)
 +
 +          Outputs information about the connected Cassandra cluster, or about
 +          the data objects stored in the cluster. Use in one of the following ways:
 +
 +        DESCRIBE KEYSPACES
 +
 +          Output the names of all keyspaces.
 +
 +        DESCRIBE KEYSPACE [<keyspacename>]
 +
 +          Output CQL commands that could be used to recreate the given keyspace,
 +          and the objects in it (such as tables, types, functions, etc.).
 +          In some cases, as the CQL interface matures, there will be some metadata
 +          about a keyspace that is not representable with CQL. That metadata will not be shown.
 +
 +          The '<keyspacename>' argument may be omitted, in which case the current
 +          keyspace will be described.
 +
 +        DESCRIBE TABLES
 +
 +          Output the names of all tables in the current keyspace, or in all
 +          keyspaces if there is no current keyspace.
 +
 +        DESCRIBE TABLE [<keyspace>.]<tablename>
 +
 +          Output CQL commands that could be used to recreate the given table.
 +          In some cases, as above, there may be table metadata which is not
 +          representable and which will not be shown.
 +
 +        DESCRIBE INDEX <indexname>
 +
 +          Output the CQL command that could be used to recreate the given index.
 +          In some cases, there may be index metadata which is not representable
 +          and which will not be shown.
 +
 +        DESCRIBE CLUSTER
 +
 +          Output information about the connected Cassandra cluster, such as the
 +          cluster name, and the partitioner and snitch in use. When you are
 +          connected to a non-system keyspace, also shows endpoint-range
 +          ownership information for the Cassandra ring.
 +
 +        DESCRIBE [FULL] SCHEMA
 +
 +          Output CQL commands that could be used to recreate the entire (non-system) schema.
 +          Works as though "DESCRIBE KEYSPACE k" was invoked for each non-system keyspace
 +          k. Use DESCRIBE FULL SCHEMA to include the system keyspaces.
 +
 +        DESCRIBE TYPES
 +
 +          Output the names of all user-defined-types in the current keyspace, or in all
 +          keyspaces if there is no current keyspace.
 +
 +        DESCRIBE TYPE [<keyspace>.]<type>
 +
 +          Output the CQL command that could be used to recreate the given user-defined-type.
 +
 +        DESCRIBE FUNCTIONS
 +
 +          Output the names of all user-defined-functions in the current keyspace, or in all
 +          keyspaces if there is no current keyspace.
 +
 +        DESCRIBE FUNCTION [<keyspace>.]<function>
 +
 +          Output the CQL command that could be used to recreate the given user-defined-function.
 +
 +        DESCRIBE AGGREGATES
 +
 +          Output the names of all user-defined-aggregates in the current keyspace, or in all
 +          keyspaces if there is no current keyspace.
 +
 +        DESCRIBE AGGREGATE [<keyspace>.]<aggregate>
 +
 +          Output the CQL command that could be used to recreate the given user-defined-aggregate.
 +
 +        DESCRIBE <objname>
 +
 +          Output CQL commands that could be used to recreate the entire object schema,
 +          where object can be either a keyspace or a table or an index (in this order).
 +  """
 +        what = parsed.matched[1][1].lower()
 +        if what == 'functions':
 +            self.describe_functions(self.current_keyspace)
 +        elif what == 'function':
 +            ksname = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            functionname = self.cql_unprotect_name(parsed.get_binding('udfname'))
 +            self.describe_function(ksname, functionname)
 +        elif what == 'aggregates':
 +            self.describe_aggregates(self.current_keyspace)
 +        elif what == 'aggregate':
 +            ksname = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            aggregatename = self.cql_unprotect_name(parsed.get_binding('udaname'))
 +            self.describe_aggregate(ksname, aggregatename)
 +        elif what == 'keyspaces':
 +            self.describe_keyspaces()
 +        elif what == 'keyspace':
 +            ksname = self.cql_unprotect_name(parsed.get_binding('ksname', ''))
 +            if not ksname:
 +                ksname = self.current_keyspace
 +                if ksname is None:
 +                    self.printerr('Not in any keyspace.')
 +                    return
 +            self.describe_keyspace(ksname)
 +        elif what in ('columnfamily', 'table'):
 +            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            cf = self.cql_unprotect_name(parsed.get_binding('cfname'))
 +            self.describe_columnfamily(ks, cf)
 +        elif what == 'index':
 +            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            idx = self.cql_unprotect_name(parsed.get_binding('idxname', None))
 +            self.describe_index(ks, idx)
 +        elif what in ('columnfamilies', 'tables'):
 +            self.describe_columnfamilies(self.current_keyspace)
 +        elif what == 'types':
 +            self.describe_usertypes(self.current_keyspace)
 +        elif what == 'type':
 +            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            ut = self.cql_unprotect_name(parsed.get_binding('utname'))
 +            self.describe_usertype(ks, ut)
 +        elif what == 'cluster':
 +            self.describe_cluster()
 +        elif what == 'schema':
 +            self.describe_schema(False)
 +        elif what == 'full' and parsed.matched[2][1].lower() == 'schema':
 +            self.describe_schema(True)
 +        elif what:
 +            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            name = self.cql_unprotect_name(parsed.get_binding('cfname'))
 +            if not name:
 +                name = self.cql_unprotect_name(parsed.get_binding('idxname', None))
 +            self.describe_object(ks, name)
 +    do_desc = do_describe
 +
 +    def do_copy(self, parsed):
 +        r"""
 +        COPY [cqlsh only]
 +
 +          COPY x FROM: Imports CSV data into a Cassandra table
 +          COPY x TO: Exports data from a Cassandra table in CSV format.
 +
 +        COPY <table_name> [ ( column [, ...] ) ]
 +             FROM ( '<file_pattern_1, file_pattern_2, ... file_pattern_n>' | STDIN )
 +             [ WITH <option>='value' [AND ...] ];
 +
 +        File patterns are either file names or valid python glob expressions, e.g. *.csv or folder/*.csv.
 +
 +        COPY <table_name> [ ( column [, ...] ) ]
 +             TO ( '<filename>' | STDOUT )
 +             [ WITH <option>='value' [AND ...] ];
 +
 +        Available common COPY options and defaults:
 +
 +          DELIMITER=','           - character that appears between records
 +          QUOTE='"'               - quoting character to be used to quote fields
 +          ESCAPE='\'              - character to appear before the QUOTE char when quoted
 +          HEADER=false            - whether to ignore the first line
 +          NULL=''                 - string that represents a null value
 +          DATETIMEFORMAT=         - timestamp strftime format
 +            '%Y-%m-%d %H:%M:%S%z'   defaults to time_format value in cqlshrc
 +          MAXATTEMPTS=5           - the maximum number of attempts per batch or range
 +          REPORTFREQUENCY=0.25    - the frequency with which we display status updates in seconds
 +          DECIMALSEP='.'          - the separator for decimal values
 +          THOUSANDSSEP=''         - the separator for thousands digit groups
 +          BOOLSTYLE='True,False'  - the representation for booleans, case insensitive, specify true followed by false,
 +                                    for example yes,no or 1,0
 +          NUMPROCESSES=n          - the number of worker processes, by default the number of cores minus one
 +                                    capped at 16
 +          CONFIGFILE=''           - a configuration file with the same format as .cqlshrc (see the Python ConfigParser
 +                                    documentation) where you can specify WITH options under the following optional
 +                                    sections: [copy], [copy-to], [copy-from], [copy:ks.table], [copy-to:ks.table],
 +                                    [copy-from:ks.table], where <ks> is your keyspace name and <table> is your table
 +                                    name. Options are read from these sections, in the order specified
 +                                    above, and command line options always override options in configuration files.
 +                                    Depending on the COPY direction, only the relevant copy-from or copy-to sections
 +                                    are used. If no configfile is specified then .cqlshrc is searched instead.
 +          RATEFILE=''             - an optional file where to print the output statistics
 +
 +        Available COPY FROM options and defaults:
 +
-           CHUNKSIZE=1000          - the size of chunks passed to worker processes
++          CHUNKSIZE=5000          - the size of chunks passed to worker processes
 +          INGESTRATE=100000       - an approximate ingest rate in rows per second
-           MINBATCHSIZE=2          - the minimum size of an import batch
++          MINBATCHSIZE=10         - the minimum size of an import batch
 +          MAXBATCHSIZE=20         - the maximum size of an import batch
 +          MAXROWS=-1              - the maximum number of rows, -1 means no maximum
 +          SKIPROWS=0              - the number of rows to skip
 +          SKIPCOLS=''             - a comma separated list of column names to skip
 +          MAXPARSEERRORS=-1       - the maximum global number of parsing errors, -1 means no maximum
 +          MAXINSERTERRORS=-1      - the maximum global number of insert errors, -1 means no maximum
 +          ERRFILE=''              - a file where to store all rows that could not be imported, by default this is
 +                                    import_ks_table.err where <ks> is your keyspace and <table> is your table name.
++          PREPAREDSTATEMENTS=True - whether to use prepared statements when importing, by default True. Set this to
++                                    False if you don't mind shifting data parsing to the cluster. The cluster will also
++                                    have to compile every batch statement. For large and oversized clusters
++                                    this will result in a faster import but for smaller clusters it may generate
++                                    timeouts.
 +
 +        Available COPY TO options and defaults:
 +
 +          ENCODING='utf8'          - encoding for CSV output
 +          PAGESIZE='1000'          - the page size for fetching results
 +          PAGETIMEOUT=10           - the page timeout in seconds for fetching results
 +          BEGINTOKEN=''            - the minimum token string to consider when exporting data
 +          ENDTOKEN=''              - the maximum token string to consider when exporting data
 +          MAXREQUESTS=6            - the maximum number of requests each worker process can work on in parallel
 +          MAXOUTPUTSIZE='-1'       - the maximum size of the output file measured in number of lines,
 +                                     beyond this maximum the output file will be split into segments,
 +                                     -1 means unlimited.
 +
 +        When entering CSV data on STDIN, you can use the sequence "\."
 +        on a line by itself to end the data input.
 +        """
 +
 +        ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +        if ks is None:
 +            ks = self.current_keyspace
 +            if ks is None:
 +                raise NoKeyspaceError("Not in any keyspace.")
 +        table = self.cql_unprotect_name(parsed.get_binding('cfname'))
 +        columns = parsed.get_binding('colnames', None)
 +        if columns is not None:
 +            columns = map(self.cql_unprotect_name, columns)
 +        else:
 +            # default to all known columns
 +            columns = self.get_column_names(ks, table)
 +
 +        fname = parsed.get_binding('fname', None)
 +        if fname is not None:
 +            fname = self.cql_unprotect_value(fname)
 +
 +        copyoptnames = map(str.lower, parsed.get_binding('optnames', ()))
 +        copyoptvals = map(self.cql_unprotect_value, parsed.get_binding('optvals', ()))
 +        opts = dict(zip(copyoptnames, copyoptvals))
 +
 +        direction = parsed.get_binding('dir').upper()
 +        if direction == 'FROM':
 +            task = ImportTask(self, ks, table, columns, fname, opts, DEFAULT_PROTOCOL_VERSION, CONFIG_FILE)
 +        elif direction == 'TO':
 +            task = ExportTask(self, ks, table, columns, fname, opts, DEFAULT_PROTOCOL_VERSION, CONFIG_FILE)
 +        else:
 +            raise SyntaxError("Unknown direction %s" % direction)
 +
 +        task.run()
 +
 +    def do_show(self, parsed):
 +        """
 +        SHOW [cqlsh only]
 +
 +          Displays information about the current cqlsh session. Can be called in
 +          the following ways:
 +
 +        SHOW VERSION
 +
 +          Shows the version and build of the connected Cassandra instance, as
 +          well as the versions of the CQL spec and the Thrift protocol that
 +          the connected Cassandra instance understands.
 +
 +        SHOW HOST
 +
 +          Shows where cqlsh is currently connected.
 +
 +        SHOW SESSION <sessionid>
 +
 +          Pretty-prints the requested tracing session.
 +        """
 +        showwhat = parsed.get_binding('what').lower()
 +        if showwhat == 'version':
 +            self.get_connection_versions()
 +            self.show_version()
 +        elif showwhat == 'host':
 +            self.show_host()
 +        elif showwhat.startswith('session'):
 +            session_id = parsed.get_binding('sessionid').lower()
 +            self.show_session(UUID(session_id))
 +        else:
 +            self.printerr('Wait, how do I show %r?' % (showwhat,))
 +
 +    def do_source(self, parsed):
 +        """
 +        SOURCE [cqlsh only]
 +
 +        Executes a file containing CQL statements. Gives the output for each
 +        statement in turn, if any, or any errors that occur along the way.
 +
 +        Errors do NOT abort execution of the CQL source file.
 +
 +        Usage:
 +
 +          SOURCE '<file>';
 +
 +        That is, the path to the file to be executed must be given inside a
 +        string literal. The path is interpreted relative to the current working
 +        directory. The tilde shorthand notation ('~/mydir') is supported for
 +        referring to $HOME.
 +
 +        See also the --file option to cqlsh.
 +        """
 +        fname = parsed.get_binding('fname')
 +        fname = os.path.expanduser(self.cql_unprotect_value(fname))
 +        try:
 +            encoding, bom_size = get_file_encoding_bomsize(fname)
 +            f = codecs.open(fname, 'r', encoding)
 +            f.seek(bom_size)
 +        except IOError, e:
 +            self.printerr('Could not open %r: %s' % (fname, e))
 +            return
 +        subshell = Shell(self.hostname, self.port,
 +                         color=self.color, encoding=self.encoding, stdin=f,
 +                         tty=False, use_conn=self.conn, cqlver=self.cql_version,
 +                         display_timestamp_format=self.display_timestamp_format,
 +                         display_date_format=self.display_date_format,
 +                         display_nanotime_format=self.display_nanotime_format,
 +                         display_float_precision=self.display_float_precision,
 +                         max_trace_wait=self.max_trace_wait)
 +        subshell.cmdloop()
 +        f.close()
 +
 +    def do_capture(self, parsed):
 +        """
 +        CAPTURE [cqlsh only]
 +
 +        Begins capturing command output and appending it to a specified file.
 +        Output will not be shown at the console while it is captured.
 +
 +        Usage:
 +
 +          CAPTURE '<file>';
 +          CAPTURE OFF;
 +          CAPTURE;
 +
 +        That is, the path to the file to be appended to must be given inside a
 +        string literal. The path is interpreted relative to the current working
 +        directory. The tilde shorthand notation ('~/mydir') is supported for
 +        referring to $HOME.
 +
 +        Only query result output is captured. Errors and output from cqlsh-only
 +        commands will still be shown in the cqlsh session.
 +
 +        To stop capturing output and show it in the cqlsh session again, use
 +        CAPTURE OFF.
 +
 +        To inspect the current capture configuration, use CAPTURE with no
 +        arguments.
 +        """
 +        fname = parsed.get_binding('fname')
 +        if fname is None:
 +            if self.shunted_query_out is not None:
 +                print "Currently capturing query output to %r." % (self.query_out.name,)
 +            else:
 +                print "Currently not capturing query output."
 +            return
 +
 +        if fname.upper() == 'OFF':
 +            if self.shunted_query_out is None:
 +                self.printerr('Not currently capturing output.')
 +                return
 +            self.query_out.close()
 +            self.query_out = self.shunted_query_out
 +            self.color = self.shunted_color
 +            self.shunted_query_out = None
 +            del self.shunted_color
 +            return
 +
 +        if self.shunted_query_out is not None:
 +            self.printerr('Already capturing output to %s. Use CAPTURE OFF'
 +                          ' to disable.' % (self.query_out.name,))
 +            return
 +
 +        fname = os.path.expanduser(self.cql_unprotect_value(fname))
 +        try:
 +            f = open(fname, 'a')
 +        except IOError, e:
 +            self.printerr('Could not open %r for append: %s' % (fname, e))
 +            return
 +        self.shunted_query_out = self.query_out
 +        self.shunted_color = self.color
 +        self.query_out = f
 +        self.color = False
 +        print 'Now capturing query output to %r.' % (fname,)
 +
 +    def do_tracing(self, parsed):
 +        """
 +        TRACING [cqlsh]
 +
 +          Enables or disables request tracing.
 +
 +        TRACING ON
 +
 +          Enables tracing for all further requests.
 +
 +        TRACING OFF
 +
 +          Disables tracing.
 +
 +        TRACING
 +
 +          TRACING with no arguments shows the current tracing status.
 +        """
 +        self.tracing_enabled = SwitchCommand("TRACING", "Tracing").execute(self.tracing_enabled, parsed, self.printerr)
 +
 +    def do_expand(self, parsed):
 +        """
 +        EXPAND [cqlsh]
 +
 +          Enables or disables expanded (vertical) output.
 +
 +        EXPAND ON
 +
 +          Enables expanded (vertical) output.
 +
 +        EXPAND OFF
 +
 +          Disables expanded (vertical) output.
 +
 +        EXPAND
 +
 +          EXPAND with no arguments shows the current value of expand setting.
 +        """
 +        self.expand_enabled = SwitchCommand("EXPAND", "Expanded output").execute(self.expand_enabled, parsed, self.printerr)
 +
 +    def do_consistency(self, parsed):
 +        """
 +        CONSISTENCY [cqlsh only]
 +
 +           Overrides default consistency level (default level is ONE).
 +
 +        CONSISTENCY <level>
 +
 +           Sets consistency level for future requests.
 +
 +           Valid consistency levels:
 +
 +           ANY, ONE, TWO, THREE, QUORUM, ALL, LOCAL_ONE, LOCAL_QUORUM, EACH_QUORUM, SERIAL and LOCAL_SERIAL.
 +
 +           SERIAL and LOCAL_SERIAL may be used only for SELECTs; will be rejected with updates.
 +
 +        CONSISTENCY
 +
 +           CONSISTENCY with no arguments shows the current consistency level.
 +        """
 +        level = parsed.get_binding('level')
 +        if level is None:
 +            print 'Current consistency level is %s.' % (cassandra.ConsistencyLevel.value_to_name[self.consistency_level])
 +            return
 +
 +        self.consistency_level = cassandra.ConsistencyLevel.name_to_value[level.upper()]
 +        print 'Consistency level set to %s.' % (level.upper(),)
 +
 +    def do_serial(self, parsed):
 +        """
 +        SERIAL CONSISTENCY [cqlsh only]
 +
 +           Overrides serial consistency level (default level is SERIAL).
 +
 +        SERIAL CONSISTENCY <level>
 +
 +           Sets consistency level for future conditional updates.
 +
 +           Valid consistency levels:
 +
 +           SERIAL, LOCAL_SERIAL.
 +
 +        SERIAL CONSISTENCY
 +
 +           SERIAL CONSISTENCY with no arguments shows the current consistency level.
 +        """
 +        level = parsed.get_binding('level')
 +        if level is None:
 +            print 'Current serial consistency level is %s.' % (cassandra.ConsistencyLevel.value_to_name[self.serial_consistency_level])
 +            return
 +
 +        self.serial_consistency_level = cassandra.ConsistencyLevel.name_to_value[level.upper()]
 +        print 'Serial consistency level set to %s.' % (level.upper(),)
 +
 +    def do_login(self, parsed):
 +        """
 +        LOGIN [cqlsh only]
 +
 +           Changes login information without requiring restart.
 +
 +        LOGIN <username> (<password>)
 +
 +           Login using the specified username. If password is specified, it will be used
 +           otherwise, you will be prompted to enter.
 +        """
 +        username = parsed.get_binding('username')
 +        password = parsed.get_binding('password')
 +        if password is None:
 +            password = getpass.getpass()
 +        else:
 +            password = password[1:-1]
 +
 +        auth_provider = PlainTextAuthProvider(username=username, password=password)
 +
 +        conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=self.conn.cql_version,
 +                       protocol_version=self.conn.protocol_version,
 +                       auth_provider=auth_provider,
 +                       ssl_options=self.conn.ssl_options,
 +                       load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]),
 +                       control_connection_timeout=self.conn.connect_timeout,
 +                       connect_timeout=self.conn.connect_timeout)
 +
 +        if self.current_keyspace:
 +            session = conn.connect(self.current_keyspace)
 +        else:
 +            session = conn.connect()
 +
 +        # Update after we've connected in case we fail to authenticate
 +        self.conn = conn
 +        self.auth_provider = auth_provider
 +        self.username = username
 +        self.session = session
 +
 +    def do_exit(self, parsed=None):
 +        """
 +        EXIT/QUIT [cqlsh only]
 +
 +        Exits cqlsh.
 +        """
 +        self.stop = True
 +        if self.owns_connection:
 +            self.conn.shutdown()
 +    do_quit = do_exit
 +
 +    def do_clear(self, parsed):
 +        """
 +        CLEAR/CLS [cqlsh only]
 +
 +        Clears the console.
 +        """
 +        import subprocess
 +        subprocess.call(['clear', 'cls'][is_win], shell=True)
 +    do_cls = do_clear
 +
 +    def do_debug(self, parsed):
 +        import pdb
 +        pdb.set_trace()
 +
 +    def get_help_topics(self):
 +        topics = [t[3:] for t in dir(self) if t.startswith('do_') and getattr(self, t, None).__doc__]
 +        for hide_from_help in ('quit',):
 +            topics.remove(hide_from_help)
 +        return topi

<TRUNCATED>

[16/23] cassandra git commit: Merge commit 'c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc' into cassandra-2.2

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b74ffeaf/bin/cqlsh.py
----------------------------------------------------------------------
diff --cc bin/cqlsh.py
index b3ec1ac,0000000..9082d72
mode 100644,000000..100644
--- a/bin/cqlsh.py
+++ b/bin/cqlsh.py
@@@ -1,2546 -1,0 +1,2566 @@@
 +#!/bin/sh
 +# -*- mode: Python -*-
 +
 +# Licensed to the Apache Software Foundation (ASF) under one
 +# or more contributor license agreements.  See the NOTICE file
 +# distributed with this work for additional information
 +# regarding copyright ownership.  The ASF licenses this file
 +# to you under the Apache License, Version 2.0 (the
 +# "License"); you may not use this file except in compliance
 +# with the License.  You may obtain a copy of the License at
 +#
 +#     http://www.apache.org/licenses/LICENSE-2.0
 +#
 +# Unless required by applicable law or agreed to in writing, software
 +# distributed under the License is distributed on an "AS IS" BASIS,
 +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 +# See the License for the specific language governing permissions and
 +# limitations under the License.
 +
 +""":"
 +# bash code here; finds a suitable python interpreter and execs this file.
 +# prefer unqualified "python" if suitable:
 +python -c 'import sys; sys.exit(not (0x020700b0 < sys.hexversion < 0x03000000))' 2>/dev/null \
 +    && exec python "$0" "$@"
 +for pyver in 2.7; do
 +    which python$pyver > /dev/null 2>&1 && exec python$pyver "$0" "$@"
 +done
 +echo "No appropriate python interpreter found." >&2
 +exit 1
 +":"""
 +
 +from __future__ import with_statement
 +
 +import cmd
 +import codecs
 +import ConfigParser
 +import csv
 +import getpass
 +import locale
 +import optparse
 +import os
 +import platform
 +import sys
 +import traceback
 +import warnings
 +import webbrowser
 +from StringIO import StringIO
 +from contextlib import contextmanager
 +from glob import glob
 +from uuid import UUID
 +
 +if sys.version_info[0] != 2 or sys.version_info[1] != 7:
 +    sys.exit("\nCQL Shell supports only Python 2.7\n")
 +
 +UTF8 = 'utf-8'
 +CP65001 = 'cp65001'  # Win utf-8 variant
 +
 +description = "CQL Shell for Apache Cassandra"
 +version = "5.0.1"
 +
 +readline = None
 +try:
 +    # check if tty first, cause readline doesn't check, and only cares
 +    # about $TERM. we don't want the funky escape code stuff to be
 +    # output if not a tty.
 +    if sys.stdin.isatty():
 +        import readline
 +except ImportError:
 +    pass
 +
 +CQL_LIB_PREFIX = 'cassandra-driver-internal-only-'
 +
 +CASSANDRA_PATH = os.path.join(os.path.dirname(os.path.realpath(__file__)), '..')
 +CASSANDRA_CQL_HTML_FALLBACK = 'https://cassandra.apache.org/doc/cql3/CQL-2.2.html'
 +
 +if os.path.exists(CASSANDRA_PATH + '/doc/cql3/CQL.html'):
 +    # default location of local CQL.html
 +    CASSANDRA_CQL_HTML = 'file://' + CASSANDRA_PATH + '/doc/cql3/CQL.html'
 +elif os.path.exists('/usr/share/doc/cassandra/CQL.html'):
 +    # fallback to package file
 +    CASSANDRA_CQL_HTML = 'file:///usr/share/doc/cassandra/CQL.html'
 +else:
 +    # fallback to online version
 +    CASSANDRA_CQL_HTML = CASSANDRA_CQL_HTML_FALLBACK
 +
 +# On Linux, the Python webbrowser module uses the 'xdg-open' executable
 +# to open a file/URL. But that only works, if the current session has been
 +# opened from _within_ a desktop environment. I.e. 'xdg-open' will fail,
 +# if the session's been opened via ssh to a remote box.
 +#
 +# Use 'python' to get some information about the detected browsers.
 +# >>> import webbrowser
 +# >>> webbrowser._tryorder
 +# >>> webbrowser._browser
 +#
 +if len(webbrowser._tryorder) == 0:
 +    CASSANDRA_CQL_HTML = CASSANDRA_CQL_HTML_FALLBACK
 +elif webbrowser._tryorder[0] == 'xdg-open' and os.environ.get('XDG_DATA_DIRS', '') == '':
 +    # only on Linux (some OS with xdg-open)
 +    webbrowser._tryorder.remove('xdg-open')
 +    webbrowser._tryorder.append('xdg-open')
 +
 +# use bundled libs for python-cql and thrift, if available. if there
 +# is a ../lib dir, use bundled libs there preferentially.
 +ZIPLIB_DIRS = [os.path.join(CASSANDRA_PATH, 'lib')]
 +myplatform = platform.system()
 +is_win = myplatform == 'Windows'
 +
 +# Workaround for supporting CP65001 encoding on python < 3.3 (https://bugs.python.org/issue13216)
 +if is_win and sys.version_info < (3, 3):
 +    codecs.register(lambda name: codecs.lookup(UTF8) if name == CP65001 else None)
 +
 +if myplatform == 'Linux':
 +    ZIPLIB_DIRS.append('/usr/share/cassandra/lib')
 +
 +if os.environ.get('CQLSH_NO_BUNDLED', ''):
 +    ZIPLIB_DIRS = ()
 +
 +
 +def find_zip(libprefix):
 +    for ziplibdir in ZIPLIB_DIRS:
 +        zips = glob(os.path.join(ziplibdir, libprefix + '*.zip'))
 +        if zips:
 +            return max(zips)   # probably the highest version, if multiple
 +
 +cql_zip = find_zip(CQL_LIB_PREFIX)
 +if cql_zip:
 +    ver = os.path.splitext(os.path.basename(cql_zip))[0][len(CQL_LIB_PREFIX):]
 +    sys.path.insert(0, os.path.join(cql_zip, 'cassandra-driver-' + ver))
 +
 +third_parties = ('futures-', 'six-')
 +
 +for lib in third_parties:
 +    lib_zip = find_zip(lib)
 +    if lib_zip:
 +        sys.path.insert(0, lib_zip)
 +
 +warnings.filterwarnings("ignore", r".*blist.*")
 +try:
 +    import cassandra
 +except ImportError, e:
 +    sys.exit("\nPython Cassandra driver not installed, or not on PYTHONPATH.\n"
 +             'You might try "pip install cassandra-driver".\n\n'
 +             'Python: %s\n'
 +             'Module load path: %r\n\n'
 +             'Error: %s\n' % (sys.executable, sys.path, e))
 +
 +from cassandra.auth import PlainTextAuthProvider
 +from cassandra.cluster import Cluster
 +from cassandra.metadata import (ColumnMetadata, KeyspaceMetadata,
 +                                TableMetadata, protect_name, protect_names)
 +from cassandra.policies import WhiteListRoundRobinPolicy
 +from cassandra.query import SimpleStatement, ordered_dict_factory, TraceUnavailable
 +
 +# cqlsh should run correctly when run out of a Cassandra source tree,
 +# out of an unpacked Cassandra tarball, and after a proper package install.
 +cqlshlibdir = os.path.join(CASSANDRA_PATH, 'pylib')
 +if os.path.isdir(cqlshlibdir):
 +    sys.path.insert(0, cqlshlibdir)
 +
 +from cqlshlib import cql3handling, cqlhandling, pylexotron, sslhandling
 +from cqlshlib.copyutil import ExportTask, ImportTask
 +from cqlshlib.displaying import (ANSI_RESET, BLUE, COLUMN_NAME_COLORS, CYAN,
 +                                 RED, FormattedValue, colorme)
 +from cqlshlib.formatting import (DEFAULT_DATE_FORMAT, DEFAULT_NANOTIME_FORMAT,
 +                                 DEFAULT_TIMESTAMP_FORMAT, DateTimeFormat,
 +                                 format_by_type, format_value_utype,
 +                                 formatter_for)
 +from cqlshlib.tracing import print_trace, print_trace_session
 +from cqlshlib.util import get_file_encoding_bomsize, trim_if_present
 +
 +DEFAULT_HOST = '127.0.0.1'
 +DEFAULT_PORT = 9042
 +DEFAULT_CQLVER = '3.3.1'
 +DEFAULT_PROTOCOL_VERSION = 4
 +DEFAULT_CONNECT_TIMEOUT_SECONDS = 5
 +DEFAULT_REQUEST_TIMEOUT_SECONDS = 10
 +
 +DEFAULT_FLOAT_PRECISION = 5
 +DEFAULT_MAX_TRACE_WAIT = 10
 +
 +if readline is not None and readline.__doc__ is not None and 'libedit' in readline.__doc__:
 +    DEFAULT_COMPLETEKEY = '\t'
 +else:
 +    DEFAULT_COMPLETEKEY = 'tab'
 +
 +cqldocs = None
 +cqlruleset = None
 +
 +epilog = """Connects to %(DEFAULT_HOST)s:%(DEFAULT_PORT)d by default. These
 +defaults can be changed by setting $CQLSH_HOST and/or $CQLSH_PORT. When a
 +host (and optional port number) are given on the command line, they take
 +precedence over any defaults.""" % globals()
 +
 +parser = optparse.OptionParser(description=description, epilog=epilog,
 +                               usage="Usage: %prog [options] [host [port]]",
 +                               version='cqlsh ' + version)
 +parser.add_option("-C", "--color", action='store_true', dest='color',
 +                  help='Always use color output')
 +parser.add_option("--no-color", action='store_false', dest='color',
 +                  help='Never use color output')
 +parser.add_option("--browser", dest='browser', help="""The browser to use to display CQL help, where BROWSER can be:
 +                                                    - one of the supported browsers in https://docs.python.org/2/library/webbrowser.html.
 +                                                    - browser path followed by %s, example: /usr/bin/google-chrome-stable %s""")
 +parser.add_option('--ssl', action='store_true', help='Use SSL', default=False)
 +parser.add_option("-u", "--username", help="Authenticate as user.")
 +parser.add_option("-p", "--password", help="Authenticate using password.")
 +parser.add_option('-k', '--keyspace', help='Authenticate to the given keyspace.')
 +parser.add_option("-f", "--file", help="Execute commands from FILE, then exit")
 +parser.add_option('--debug', action='store_true',
 +                  help='Show additional debugging information')
 +parser.add_option("--encoding", help="Specify a non-default encoding for output." +
 +                  " (Default: %s)" % (UTF8,))
 +parser.add_option("--cqlshrc", help="Specify an alternative cqlshrc file location.")
 +parser.add_option('--cqlversion', default=DEFAULT_CQLVER,
 +                  help='Specify a particular CQL version (default: %default).'
 +                       ' Examples: "3.0.3", "3.1.0"')
 +parser.add_option("-e", "--execute", help='Execute the statement and quit.')
 +parser.add_option("--connect-timeout", default=DEFAULT_CONNECT_TIMEOUT_SECONDS, dest='connect_timeout',
 +                  help='Specify the connection timeout in seconds (default: %default seconds).')
 +parser.add_option("--request-timeout", default=DEFAULT_REQUEST_TIMEOUT_SECONDS, dest='request_timeout',
 +                  help='Specify the default request timeout in seconds (default: %default seconds).')
 +parser.add_option("-t", "--tty", action='store_true', dest='tty',
 +                  help='Force tty mode (command prompt).')
 +
 +optvalues = optparse.Values()
 +(options, arguments) = parser.parse_args(sys.argv[1:], values=optvalues)
 +
 +# BEGIN history/config definition
 +HISTORY_DIR = os.path.expanduser(os.path.join('~', '.cassandra'))
 +
 +if hasattr(options, 'cqlshrc'):
 +    CONFIG_FILE = options.cqlshrc
 +    if not os.path.exists(CONFIG_FILE):
 +        print '\nWarning: Specified cqlshrc location `%s` does not exist.  Using `%s` instead.\n' % (CONFIG_FILE, HISTORY_DIR)
 +        CONFIG_FILE = os.path.join(HISTORY_DIR, 'cqlshrc')
 +else:
 +    CONFIG_FILE = os.path.join(HISTORY_DIR, 'cqlshrc')
 +
 +HISTORY = os.path.join(HISTORY_DIR, 'cqlsh_history')
 +if not os.path.exists(HISTORY_DIR):
 +    try:
 +        os.mkdir(HISTORY_DIR)
 +    except OSError:
 +        print '\nWarning: Cannot create directory at `%s`. Command history will not be saved.\n' % HISTORY_DIR
 +
 +OLD_CONFIG_FILE = os.path.expanduser(os.path.join('~', '.cqlshrc'))
 +if os.path.exists(OLD_CONFIG_FILE):
 +    if os.path.exists(CONFIG_FILE):
 +        print '\nWarning: cqlshrc config files were found at both the old location (%s) and \
 +                the new location (%s), the old config file will not be migrated to the new \
 +                location, and the new location will be used for now.  You should manually \
 +                consolidate the config files at the new location and remove the old file.' \
 +                % (OLD_CONFIG_FILE, CONFIG_FILE)
 +    else:
 +        os.rename(OLD_CONFIG_FILE, CONFIG_FILE)
 +OLD_HISTORY = os.path.expanduser(os.path.join('~', '.cqlsh_history'))
 +if os.path.exists(OLD_HISTORY):
 +    os.rename(OLD_HISTORY, HISTORY)
 +# END history/config definition
 +
 +CQL_ERRORS = (
 +    cassandra.AlreadyExists, cassandra.AuthenticationFailed, cassandra.InvalidRequest,
 +    cassandra.Timeout, cassandra.Unauthorized, cassandra.OperationTimedOut,
 +    cassandra.cluster.NoHostAvailable,
 +    cassandra.connection.ConnectionBusy, cassandra.connection.ProtocolError, cassandra.connection.ConnectionException,
 +    cassandra.protocol.ErrorMessage, cassandra.protocol.InternalError, cassandra.query.TraceUnavailable
 +)
 +
 +debug_completion = bool(os.environ.get('CQLSH_DEBUG_COMPLETION', '') == 'YES')
 +
 +# we want the cql parser to understand our cqlsh-specific commands too
 +my_commands_ending_with_newline = (
 +    'help',
 +    '?',
 +    'consistency',
 +    'serial',
 +    'describe',
 +    'desc',
 +    'show',
 +    'source',
 +    'capture',
 +    'login',
 +    'debug',
 +    'tracing',
 +    'expand',
 +    'paging',
 +    'exit',
 +    'quit',
 +    'clear',
 +    'cls'
 +)
 +
 +
 +cqlsh_syntax_completers = []
 +
 +
 +def cqlsh_syntax_completer(rulename, termname):
 +    def registrator(f):
 +        cqlsh_syntax_completers.append((rulename, termname, f))
 +        return f
 +    return registrator
 +
 +
 +cqlsh_extra_syntax_rules = r'''
 +<cqlshCommand> ::= <CQL_Statement>
 +                 | <specialCommand> ( ";" | "\n" )
 +                 ;
 +
 +<specialCommand> ::= <describeCommand>
 +                   | <consistencyCommand>
 +                   | <serialConsistencyCommand>
 +                   | <showCommand>
 +                   | <sourceCommand>
 +                   | <captureCommand>
 +                   | <copyCommand>
 +                   | <loginCommand>
 +                   | <debugCommand>
 +                   | <helpCommand>
 +                   | <tracingCommand>
 +                   | <expandCommand>
 +                   | <exitCommand>
 +                   | <pagingCommand>
 +                   | <clearCommand>
 +                   ;
 +
 +<describeCommand> ::= ( "DESCRIBE" | "DESC" )
 +                                  ( "FUNCTIONS"
 +                                  | "FUNCTION" udf=<anyFunctionName>
 +                                  | "AGGREGATES"
 +                                  | "AGGREGATE" uda=<userAggregateName>
 +                                  | "KEYSPACES"
 +                                  | "KEYSPACE" ksname=<keyspaceName>?
 +                                  | ( "COLUMNFAMILY" | "TABLE" ) cf=<columnFamilyName>
 +                                  | "INDEX" idx=<indexName>
 +                                  | ( "COLUMNFAMILIES" | "TABLES" )
 +                                  | "FULL"? "SCHEMA"
 +                                  | "CLUSTER"
 +                                  | "TYPES"
 +                                  | "TYPE" ut=<userTypeName>
 +                                  | (ksname=<keyspaceName> | cf=<columnFamilyName> | idx=<indexName>))
 +                    ;
 +
 +<consistencyCommand> ::= "CONSISTENCY" ( level=<consistencyLevel> )?
 +                       ;
 +
 +<consistencyLevel> ::= "ANY"
 +                     | "ONE"
 +                     | "TWO"
 +                     | "THREE"
 +                     | "QUORUM"
 +                     | "ALL"
 +                     | "LOCAL_QUORUM"
 +                     | "EACH_QUORUM"
 +                     | "SERIAL"
 +                     | "LOCAL_SERIAL"
 +                     | "LOCAL_ONE"
 +                     ;
 +
 +<serialConsistencyCommand> ::= "SERIAL" "CONSISTENCY" ( level=<serialConsistencyLevel> )?
 +                             ;
 +
 +<serialConsistencyLevel> ::= "SERIAL"
 +                           | "LOCAL_SERIAL"
 +                           ;
 +
 +<showCommand> ::= "SHOW" what=( "VERSION" | "HOST" | "SESSION" sessionid=<uuid> )
 +                ;
 +
 +<sourceCommand> ::= "SOURCE" fname=<stringLiteral>
 +                  ;
 +
 +<captureCommand> ::= "CAPTURE" ( fname=( <stringLiteral> | "OFF" ) )?
 +                   ;
 +
 +<copyCommand> ::= "COPY" cf=<columnFamilyName>
 +                         ( "(" [colnames]=<colname> ( "," [colnames]=<colname> )* ")" )?
 +                         ( dir="FROM" ( fname=<stringLiteral> | "STDIN" )
 +                         | dir="TO"   ( fname=<stringLiteral> | "STDOUT" ) )
 +                         ( "WITH" <copyOption> ( "AND" <copyOption> )* )?
 +                ;
 +
 +<copyOption> ::= [optnames]=(<identifier>|<reserved_identifier>) "=" [optvals]=<copyOptionVal>
 +               ;
 +
 +<copyOptionVal> ::= <identifier>
 +                  | <reserved_identifier>
 +                  | <term>
 +                  ;
 +
 +# avoiding just "DEBUG" so that this rule doesn't get treated as a terminal
 +<debugCommand> ::= "DEBUG" "THINGS"?
 +                 ;
 +
 +<helpCommand> ::= ( "HELP" | "?" ) [topic]=( /[a-z_]*/ )*
 +                ;
 +
 +<tracingCommand> ::= "TRACING" ( switch=( "ON" | "OFF" ) )?
 +                   ;
 +
 +<expandCommand> ::= "EXPAND" ( switch=( "ON" | "OFF" ) )?
 +                   ;
 +
 +<pagingCommand> ::= "PAGING" ( switch=( "ON" | "OFF" | /[0-9]+/) )?
 +                  ;
 +
 +<loginCommand> ::= "LOGIN" username=<username> (password=<stringLiteral>)?
 +                 ;
 +
 +<exitCommand> ::= "exit" | "quit"
 +                ;
 +
 +<clearCommand> ::= "CLEAR" | "CLS"
 +                 ;
 +
 +<qmark> ::= "?" ;
 +'''
 +
 +
 +@cqlsh_syntax_completer('helpCommand', 'topic')
 +def complete_help(ctxt, cqlsh):
 +    return sorted([t.upper() for t in cqldocs.get_help_topics() + cqlsh.get_help_topics()])
 +
 +
 +def complete_source_quoted_filename(ctxt, cqlsh):
 +    partial_path = ctxt.get_binding('partial', '')
 +    head, tail = os.path.split(partial_path)
 +    exhead = os.path.expanduser(head)
 +    try:
 +        contents = os.listdir(exhead or '.')
 +    except OSError:
 +        return ()
 +    matches = filter(lambda f: f.startswith(tail), contents)
 +    annotated = []
 +    for f in matches:
 +        match = os.path.join(head, f)
 +        if os.path.isdir(os.path.join(exhead, f)):
 +            match += '/'
 +        annotated.append(match)
 +    return annotated
 +
 +
 +cqlsh_syntax_completer('sourceCommand', 'fname')(complete_source_quoted_filename)
 +cqlsh_syntax_completer('captureCommand', 'fname')(complete_source_quoted_filename)
 +
 +
 +@cqlsh_syntax_completer('copyCommand', 'fname')
 +def copy_fname_completer(ctxt, cqlsh):
 +    lasttype = ctxt.get_binding('*LASTTYPE*')
 +    if lasttype == 'unclosedString':
 +        return complete_source_quoted_filename(ctxt, cqlsh)
 +    partial_path = ctxt.get_binding('partial')
 +    if partial_path == '':
 +        return ["'"]
 +    return ()
 +
 +
 +@cqlsh_syntax_completer('copyCommand', 'colnames')
 +def complete_copy_column_names(ctxt, cqlsh):
 +    existcols = map(cqlsh.cql_unprotect_name, ctxt.get_binding('colnames', ()))
 +    ks = cqlsh.cql_unprotect_name(ctxt.get_binding('ksname', None))
 +    cf = cqlsh.cql_unprotect_name(ctxt.get_binding('cfname'))
 +    colnames = cqlsh.get_column_names(ks, cf)
 +    if len(existcols) == 0:
 +        return [colnames[0]]
 +    return set(colnames[1:]) - set(existcols)
 +
 +
 +COPY_COMMON_OPTIONS = ['DELIMITER', 'QUOTE', 'ESCAPE', 'HEADER', 'NULL', 'DATETIMEFORMAT',
 +                       'MAXATTEMPTS', 'REPORTFREQUENCY', 'DECIMALSEP', 'THOUSANDSSEP', 'BOOLSTYLE',
 +                       'NUMPROCESSES', 'CONFIGFILE', 'RATEFILE']
 +COPY_FROM_OPTIONS = ['CHUNKSIZE', 'INGESTRATE', 'MAXBATCHSIZE', 'MINBATCHSIZE', 'MAXROWS',
-                      'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE']
++                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE', 'PREPAREDSTATEMENTS']
 +COPY_TO_OPTIONS = ['ENCODING', 'PAGESIZE', 'PAGETIMEOUT', 'BEGINTOKEN', 'ENDTOKEN', 'MAXOUTPUTSIZE', 'MAXREQUESTS']
 +
 +
 +@cqlsh_syntax_completer('copyOption', 'optnames')
 +def complete_copy_options(ctxt, cqlsh):
 +    optnames = map(str.upper, ctxt.get_binding('optnames', ()))
 +    direction = ctxt.get_binding('dir').upper()
 +    if direction == 'FROM':
 +        opts = set(COPY_COMMON_OPTIONS + COPY_FROM_OPTIONS) - set(optnames)
 +    elif direction == 'TO':
 +        opts = set(COPY_COMMON_OPTIONS + COPY_TO_OPTIONS) - set(optnames)
 +    return opts
 +
 +
 +@cqlsh_syntax_completer('copyOption', 'optvals')
 +def complete_copy_opt_values(ctxt, cqlsh):
 +    optnames = ctxt.get_binding('optnames', ())
 +    lastopt = optnames[-1].lower()
 +    if lastopt == 'header':
 +        return ['true', 'false']
 +    return [cqlhandling.Hint('<single_character_string>')]
 +
 +
 +class NoKeyspaceError(Exception):
 +    pass
 +
 +
 +class KeyspaceNotFound(Exception):
 +    pass
 +
 +
 +class ColumnFamilyNotFound(Exception):
 +    pass
 +
 +
 +class IndexNotFound(Exception):
 +    pass
 +
 +
 +class ObjectNotFound(Exception):
 +    pass
 +
 +
 +class VersionNotSupported(Exception):
 +    pass
 +
 +
 +class UserTypeNotFound(Exception):
 +    pass
 +
 +
 +class FunctionNotFound(Exception):
 +    pass
 +
 +
 +class AggregateNotFound(Exception):
 +    pass
 +
 +
 +class DecodeError(Exception):
 +    verb = 'decode'
 +
 +    def __init__(self, thebytes, err, colname=None):
 +        self.thebytes = thebytes
 +        self.err = err
 +        self.colname = colname
 +
 +    def __str__(self):
 +        return str(self.thebytes)
 +
 +    def message(self):
 +        what = 'value %r' % (self.thebytes,)
 +        if self.colname is not None:
 +            what = 'value %r (for column %r)' % (self.thebytes, self.colname)
 +        return 'Failed to %s %s : %s' \
 +               % (self.verb, what, self.err)
 +
 +    def __repr__(self):
 +        return '<%s %s>' % (self.__class__.__name__, self.message())
 +
 +
 +class FormatError(DecodeError):
 +    verb = 'format'
 +
 +
 +def full_cql_version(ver):
 +    while ver.count('.') < 2:
 +        ver += '.0'
 +    ver_parts = ver.split('-', 1) + ['']
 +    vertuple = tuple(map(int, ver_parts[0].split('.')) + [ver_parts[1]])
 +    return ver, vertuple
 +
 +
 +def format_value(val, output_encoding, addcolor=False, date_time_format=None,
 +                 float_precision=None, colormap=None, nullval=None):
 +    if isinstance(val, DecodeError):
 +        if addcolor:
 +            return colorme(repr(val.thebytes), colormap, 'error')
 +        else:
 +            return FormattedValue(repr(val.thebytes))
 +    return format_by_type(type(val), val, output_encoding, colormap=colormap,
 +                          addcolor=addcolor, nullval=nullval, date_time_format=date_time_format,
 +                          float_precision=float_precision)
 +
 +
 +def show_warning_without_quoting_line(message, category, filename, lineno, file=None, line=None):
 +    if file is None:
 +        file = sys.stderr
 +    try:
 +        file.write(warnings.formatwarning(message, category, filename, lineno, line=''))
 +    except IOError:
 +        pass
 +warnings.showwarning = show_warning_without_quoting_line
 +warnings.filterwarnings('always', category=cql3handling.UnexpectedTableStructure)
 +
 +
 +def insert_driver_hooks():
 +    extend_cql_deserialization()
 +    auto_format_udts()
 +
 +
 +def extend_cql_deserialization():
 +    """
-     The python driver returns BLOBs as string, but we expect them as bytearrays
++    The python driver returns BLOBs as string, but we expect them as bytearrays; therefore we change
++    the implementation of cassandra.cqltypes.BytesType.deserialize.
++
++    The deserializers package exists only when the driver has been compiled with cython extensions and
++    cassandra.deserializers.DesBytesType replaces cassandra.cqltypes.BytesType.deserialize.
++
++    DesBytesTypeByteArray is a fast deserializer that converts blobs into bytearrays but it was
++    only introduced recently (3.1.0). If it is available we use it, otherwise we remove
++    cassandra.deserializers.DesBytesType so that we fall back onto cassandra.cqltypes.BytesType.deserialize
++    just like in the case where no cython extensions are present.
 +    """
++    if hasattr(cassandra, 'deserializers'):
++        if hasattr(cassandra.deserializers, 'DesBytesTypeByteArray'):
++            cassandra.deserializers.DesBytesType = cassandra.deserializers.DesBytesTypeByteArray
++        else:
++            del cassandra.deserializers.DesBytesType
++
 +    cassandra.cqltypes.BytesType.deserialize = staticmethod(lambda byts, protocol_version: bytearray(byts))
 +    cassandra.cqltypes.CassandraType.support_empty_values = True
 +
 +
 +def auto_format_udts():
 +    # when we see a new user defined type, set up the shell formatting for it
 +    udt_apply_params = cassandra.cqltypes.UserType.apply_parameters
 +
 +    def new_apply_params(cls, *args, **kwargs):
 +        udt_class = udt_apply_params(*args, **kwargs)
 +        formatter_for(udt_class.typename)(format_value_utype)
 +        return udt_class
 +
 +    cassandra.cqltypes.UserType.udt_apply_parameters = classmethod(new_apply_params)
 +
 +    make_udt_class = cassandra.cqltypes.UserType.make_udt_class
 +
 +    def new_make_udt_class(cls, *args, **kwargs):
 +        udt_class = make_udt_class(*args, **kwargs)
 +        formatter_for(udt_class.tuple_type.__name__)(format_value_utype)
 +        return udt_class
 +
 +    cassandra.cqltypes.UserType.make_udt_class = classmethod(new_make_udt_class)
 +
 +
 +class FrozenType(cassandra.cqltypes._ParameterizedType):
 +    """
 +    Needed until the bundled python driver adds FrozenType.
 +    """
 +    typename = "frozen"
 +    num_subtypes = 1
 +
 +    @classmethod
 +    def deserialize_safe(cls, byts, protocol_version):
 +        subtype, = cls.subtypes
 +        return subtype.from_binary(byts)
 +
 +    @classmethod
 +    def serialize_safe(cls, val, protocol_version):
 +        subtype, = cls.subtypes
 +        return subtype.to_binary(val, protocol_version)
 +
 +
 +class Shell(cmd.Cmd):
 +    custom_prompt = os.getenv('CQLSH_PROMPT', '')
 +    if custom_prompt is not '':
 +        custom_prompt += "\n"
 +    default_prompt = custom_prompt + "cqlsh> "
 +    continue_prompt = "   ... "
 +    keyspace_prompt = custom_prompt + "cqlsh:%s> "
 +    keyspace_continue_prompt = "%s    ... "
 +    show_line_nums = False
 +    debug = False
 +    stop = False
 +    last_hist = None
 +    shunted_query_out = None
 +    use_paging = True
 +
 +    default_page_size = 100
 +
 +    def __init__(self, hostname, port, color=False,
 +                 username=None, password=None, encoding=None, stdin=None, tty=True,
 +                 completekey=DEFAULT_COMPLETEKEY, browser=None, use_conn=None,
 +                 cqlver=DEFAULT_CQLVER, keyspace=None,
 +                 tracing_enabled=False, expand_enabled=False,
 +                 display_nanotime_format=DEFAULT_NANOTIME_FORMAT,
 +                 display_timestamp_format=DEFAULT_TIMESTAMP_FORMAT,
 +                 display_date_format=DEFAULT_DATE_FORMAT,
 +                 display_float_precision=DEFAULT_FLOAT_PRECISION,
 +                 display_timezone=None,
 +                 max_trace_wait=DEFAULT_MAX_TRACE_WAIT,
 +                 ssl=False,
 +                 single_statement=None,
 +                 request_timeout=DEFAULT_REQUEST_TIMEOUT_SECONDS,
 +                 protocol_version=DEFAULT_PROTOCOL_VERSION,
 +                 connect_timeout=DEFAULT_CONNECT_TIMEOUT_SECONDS):
 +        cmd.Cmd.__init__(self, completekey=completekey)
 +        self.hostname = hostname
 +        self.port = port
 +        self.auth_provider = None
 +        if username:
 +            if not password:
 +                password = getpass.getpass()
 +            self.auth_provider = PlainTextAuthProvider(username=username, password=password)
 +        self.username = username
 +        self.keyspace = keyspace
 +        self.ssl = ssl
 +        self.tracing_enabled = tracing_enabled
 +        self.page_size = self.default_page_size
 +        self.expand_enabled = expand_enabled
 +        if use_conn:
 +            self.conn = use_conn
 +        else:
 +            self.conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=cqlver,
 +                                protocol_version=protocol_version,
 +                                auth_provider=self.auth_provider,
 +                                ssl_options=sslhandling.ssl_settings(hostname, CONFIG_FILE) if ssl else None,
 +                                load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]),
 +                                control_connection_timeout=connect_timeout,
 +                                connect_timeout=connect_timeout)
 +        self.owns_connection = not use_conn
 +        self.set_expanded_cql_version(cqlver)
 +
 +        if keyspace:
 +            self.session = self.conn.connect(keyspace)
 +        else:
 +            self.session = self.conn.connect()
 +
 +        if browser == "":
 +            browser = None
 +        self.browser = browser
 +        self.color = color
 +
 +        self.display_nanotime_format = display_nanotime_format
 +        self.display_timestamp_format = display_timestamp_format
 +        self.display_date_format = display_date_format
 +
 +        self.display_float_precision = display_float_precision
 +
 +        self.display_timezone = display_timezone
 +
 +        # If there is no schema metadata present (due to a schema mismatch), force schema refresh
 +        if not self.conn.metadata.keyspaces:
 +            self.refresh_schema_metadata_best_effort()
 +
 +        self.session.default_timeout = request_timeout
 +        self.session.row_factory = ordered_dict_factory
 +        self.session.default_consistency_level = cassandra.ConsistencyLevel.ONE
 +        self.get_connection_versions()
 +
 +        self.current_keyspace = keyspace
 +
 +        self.display_timestamp_format = display_timestamp_format
 +        self.display_nanotime_format = display_nanotime_format
 +        self.display_date_format = display_date_format
 +
 +        self.max_trace_wait = max_trace_wait
 +        self.session.max_trace_wait = max_trace_wait
 +
 +        self.tty = tty
 +        self.encoding = encoding
 +        self.check_windows_encoding()
 +
 +        self.output_codec = codecs.lookup(encoding)
 +
 +        self.statement = StringIO()
 +        self.lineno = 1
 +        self.in_comment = False
 +
 +        self.prompt = ''
 +        if stdin is None:
 +            stdin = sys.stdin
 +
 +        if tty:
 +            self.reset_prompt()
 +            self.report_connection()
 +            print 'Use HELP for help.'
 +        else:
 +            self.show_line_nums = True
 +        self.stdin = stdin
 +        self.query_out = sys.stdout
 +        self.consistency_level = cassandra.ConsistencyLevel.ONE
 +        self.serial_consistency_level = cassandra.ConsistencyLevel.SERIAL
 +
 +        self.empty_lines = 0
 +        self.statement_error = False
 +        self.single_statement = single_statement
 +
 +    @property
 +    def is_using_utf8(self):
 +        # utf8 encodings from https://docs.python.org/{2,3}/library/codecs.html
 +        return self.encoding.replace('-', '_').lower() in ['utf', 'utf_8', 'u8', 'utf8', CP65001]
 +
 +    def check_windows_encoding(self):
 +        if is_win and os.name == 'nt' and self.tty and \
 +           self.is_using_utf8 and sys.stdout.encoding != CP65001:
 +            self.printerr("\nWARNING: console codepage must be set to cp65001 "
 +                          "to support {} encoding on Windows platforms.\n"
 +                          "If you experience encoding problems, change your console"
 +                          " codepage with 'chcp 65001' before starting cqlsh.\n".format(self.encoding))
 +
 +    def refresh_schema_metadata_best_effort(self):
 +        try:
 +            self.conn.refresh_schema_metadata(5)  # will throw exception if there is a schema mismatch
 +        except Exception:
 +            self.printerr("Warning: schema version mismatch detected, which might be caused by DOWN nodes; if "
 +                          "this is not the case, check the schema versions of your nodes in system.local and "
 +                          "system.peers.")
 +            self.conn.refresh_schema_metadata(0)
 +
 +    def set_expanded_cql_version(self, ver):
 +        ver, vertuple = full_cql_version(ver)
 +        self.cql_version = ver
 +        self.cql_ver_tuple = vertuple
 +
 +    def cqlver_atleast(self, major, minor=0, patch=0):
 +        return self.cql_ver_tuple[:3] >= (major, minor, patch)
 +
 +    def myformat_value(self, val, **kwargs):
 +        if isinstance(val, DecodeError):
 +            self.decoding_errors.append(val)
 +        try:
 +            dtformats = DateTimeFormat(timestamp_format=self.display_timestamp_format,
 +                                       date_format=self.display_date_format, nanotime_format=self.display_nanotime_format,
 +                                       timezone=self.display_timezone)
 +            return format_value(val, self.output_codec.name,
 +                                addcolor=self.color, date_time_format=dtformats,
 +                                float_precision=self.display_float_precision, **kwargs)
 +        except Exception, e:
 +            err = FormatError(val, e)
 +            self.decoding_errors.append(err)
 +            return format_value(err, self.output_codec.name, addcolor=self.color)
 +
 +    def myformat_colname(self, name, table_meta=None):
 +        column_colors = COLUMN_NAME_COLORS.copy()
 +        # check column role and color appropriately
 +        if table_meta:
 +            if name in [col.name for col in table_meta.partition_key]:
 +                column_colors.default_factory = lambda: RED
 +            elif name in [col.name for col in table_meta.clustering_key]:
 +                column_colors.default_factory = lambda: CYAN
 +        return self.myformat_value(name, colormap=column_colors)
 +
 +    def report_connection(self):
 +        self.show_host()
 +        self.show_version()
 +
 +    def show_host(self):
 +        print "Connected to %s at %s:%d." % \
 +            (self.applycolor(self.get_cluster_name(), BLUE),
 +              self.hostname,
 +              self.port)
 +
 +    def show_version(self):
 +        vers = self.connection_versions.copy()
 +        vers['shver'] = version
 +        # system.Versions['cql'] apparently does not reflect changes with
 +        # set_cql_version.
 +        vers['cql'] = self.cql_version
 +        print "[cqlsh %(shver)s | Cassandra %(build)s | CQL spec %(cql)s | Native protocol v%(protocol)s]" % vers
 +
 +    def show_session(self, sessionid, partial_session=False):
 +        print_trace_session(self, self.session, sessionid, partial_session)
 +
 +    def get_connection_versions(self):
 +        result, = self.session.execute("select * from system.local where key = 'local'")
 +        vers = {
 +            'build': result['release_version'],
 +            'protocol': result['native_protocol_version'],
 +            'cql': result['cql_version'],
 +        }
 +        self.connection_versions = vers
 +
 +    def get_keyspace_names(self):
 +        return map(str, self.conn.metadata.keyspaces.keys())
 +
 +    def get_columnfamily_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return map(str, self.get_keyspace_meta(ksname).tables.keys())
 +
 +    def get_index_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return map(str, self.get_keyspace_meta(ksname).indexes.keys())
 +
 +    def get_column_names(self, ksname, cfname):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        layout = self.get_table_meta(ksname, cfname)
 +        return [str(col) for col in layout.columns]
 +
 +    def get_usertype_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return self.get_keyspace_meta(ksname).user_types.keys()
 +
 +    def get_usertype_layout(self, ksname, typename):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        ks_meta = self.get_keyspace_meta(ksname)
 +
 +        try:
 +            user_type = ks_meta.user_types[typename]
 +        except KeyError:
 +            raise UserTypeNotFound("User type %r not found" % typename)
 +
 +        return [(field_name, field_type.cql_parameterized_type())
 +                for field_name, field_type in zip(user_type.field_names, user_type.field_types)]
 +
 +    def get_userfunction_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return map(lambda f: f.name, self.get_keyspace_meta(ksname).functions.values())
 +
 +    def get_useraggregate_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return map(lambda f: f.name, self.get_keyspace_meta(ksname).aggregates.values())
 +
 +    def get_cluster_name(self):
 +        return self.conn.metadata.cluster_name
 +
 +    def get_partitioner(self):
 +        return self.conn.metadata.partitioner
 +
 +    def get_keyspace_meta(self, ksname):
 +        if ksname not in self.conn.metadata.keyspaces:
 +            raise KeyspaceNotFound('Keyspace %r not found.' % ksname)
 +        return self.conn.metadata.keyspaces[ksname]
 +
 +    def get_keyspaces(self):
 +        return self.conn.metadata.keyspaces.values()
 +
 +    def get_ring(self, ks):
 +        self.conn.metadata.token_map.rebuild_keyspace(ks, build_if_absent=True)
 +        return self.conn.metadata.token_map.tokens_to_hosts_by_ks[ks]
 +
 +    def get_table_meta(self, ksname, tablename):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        ksmeta = self.get_keyspace_meta(ksname)
 +
 +        if tablename not in ksmeta.tables:
 +            if ksname == 'system_auth' and tablename in ['roles', 'role_permissions']:
 +                self.get_fake_auth_table_meta(ksname, tablename)
 +            else:
 +                raise ColumnFamilyNotFound("Column family %r not found" % tablename)
 +        else:
 +            return ksmeta.tables[tablename]
 +
 +    def get_fake_auth_table_meta(self, ksname, tablename):
 +        # may be using external auth implementation so internal tables
 +        # aren't actually defined in schema. In this case, we'll fake
 +        # them up
 +        if tablename == 'roles':
 +            ks_meta = KeyspaceMetadata(ksname, True, None, None)
 +            table_meta = TableMetadata(ks_meta, 'roles')
 +            table_meta.columns['role'] = ColumnMetadata(table_meta, 'role', cassandra.cqltypes.UTF8Type)
 +            table_meta.columns['is_superuser'] = ColumnMetadata(table_meta, 'is_superuser', cassandra.cqltypes.BooleanType)
 +            table_meta.columns['can_login'] = ColumnMetadata(table_meta, 'can_login', cassandra.cqltypes.BooleanType)
 +        elif tablename == 'role_permissions':
 +            ks_meta = KeyspaceMetadata(ksname, True, None, None)
 +            table_meta = TableMetadata(ks_meta, 'role_permissions')
 +            table_meta.columns['role'] = ColumnMetadata(table_meta, 'role', cassandra.cqltypes.UTF8Type)
 +            table_meta.columns['resource'] = ColumnMetadata(table_meta, 'resource', cassandra.cqltypes.UTF8Type)
 +            table_meta.columns['permission'] = ColumnMetadata(table_meta, 'permission', cassandra.cqltypes.UTF8Type)
 +        else:
 +            raise ColumnFamilyNotFound("Column family %r not found" % tablename)
 +
 +    def get_index_meta(self, ksname, idxname):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        ksmeta = self.get_keyspace_meta(ksname)
 +
 +        if idxname not in ksmeta.indexes:
 +            raise IndexNotFound("Index %r not found" % idxname)
 +
 +        return ksmeta.indexes[idxname]
 +
 +    def get_object_meta(self, ks, name):
 +        if name is None:
 +            if ks and ks in self.conn.metadata.keyspaces:
 +                return self.conn.metadata.keyspaces[ks]
 +            elif self.current_keyspace is None:
 +                raise ObjectNotFound("%r not found in keyspaces" % (ks))
 +            else:
 +                name = ks
 +                ks = self.current_keyspace
 +
 +        if ks is None:
 +            ks = self.current_keyspace
 +
 +        ksmeta = self.get_keyspace_meta(ks)
 +
 +        if name in ksmeta.tables:
 +            return ksmeta.tables[name]
 +        elif name in ksmeta.indexes:
 +            return ksmeta.indexes[name]
 +
 +        raise ObjectNotFound("%r not found in keyspace %r" % (name, ks))
 +
 +    def get_usertypes_meta(self):
 +        data = self.session.execute("select * from system.schema_usertypes")
 +        if not data:
 +            return cql3handling.UserTypesMeta({})
 +
 +        return cql3handling.UserTypesMeta.from_layout(data)
 +
 +    def get_trigger_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return [trigger.name
 +                for table in self.get_keyspace_meta(ksname).tables.values()
 +                for trigger in table.triggers.values()]
 +
 +    def reset_statement(self):
 +        self.reset_prompt()
 +        self.statement.truncate(0)
 +        self.empty_lines = 0
 +
 +    def reset_prompt(self):
 +        if self.current_keyspace is None:
 +            self.set_prompt(self.default_prompt, True)
 +        else:
 +            self.set_prompt(self.keyspace_prompt % self.current_keyspace, True)
 +
 +    def set_continue_prompt(self):
 +        if self.empty_lines >= 3:
 +            self.set_prompt("Statements are terminated with a ';'.  You can press CTRL-C to cancel an incomplete statement.")
 +            self.empty_lines = 0
 +            return
 +        if self.current_keyspace is None:
 +            self.set_prompt(self.continue_prompt)
 +        else:
 +            spaces = ' ' * len(str(self.current_keyspace))
 +            self.set_prompt(self.keyspace_continue_prompt % spaces)
 +        self.empty_lines = self.empty_lines + 1 if not self.lastcmd else 0
 +
 +    @contextmanager
 +    def prepare_loop(self):
 +        readline = None
 +        if self.tty and self.completekey:
 +            try:
 +                import readline
 +            except ImportError:
 +                if is_win:
 +                    print "WARNING: pyreadline dependency missing.  Install to enable tab completion."
 +                pass
 +            else:
 +                old_completer = readline.get_completer()
 +                readline.set_completer(self.complete)
 +                if readline.__doc__ is not None and 'libedit' in readline.__doc__:
 +                    readline.parse_and_bind("bind -e")
 +                    readline.parse_and_bind("bind '" + self.completekey + "' rl_complete")
 +                    readline.parse_and_bind("bind ^R em-inc-search-prev")
 +                else:
 +                    readline.parse_and_bind(self.completekey + ": complete")
 +        try:
 +            yield
 +        finally:
 +            if readline is not None:
 +                readline.set_completer(old_completer)
 +
 +    def get_input_line(self, prompt=''):
 +        if self.tty:
 +            try:
 +                self.lastcmd = raw_input(prompt).decode(self.encoding)
 +            except UnicodeDecodeError:
 +                self.lastcmd = ''
 +                traceback.print_exc()
 +                self.check_windows_encoding()
 +            line = self.lastcmd + '\n'
 +        else:
 +            self.lastcmd = self.stdin.readline()
 +            line = self.lastcmd
 +            if not len(line):
 +                raise EOFError
 +        self.lineno += 1
 +        return line
 +
 +    def use_stdin_reader(self, until='', prompt=''):
 +        until += '\n'
 +        while True:
 +            try:
 +                newline = self.get_input_line(prompt=prompt)
 +            except EOFError:
 +                return
 +            if newline == until:
 +                return
 +            yield newline
 +
 +    def cmdloop(self):
 +        """
 +        Adapted from cmd.Cmd's version, because there is literally no way with
 +        cmd.Cmd.cmdloop() to tell the difference between "EOF" showing up in
 +        input and an actual EOF.
 +        """
 +        with self.prepare_loop():
 +            while not self.stop:
 +                try:
 +                    if self.single_statement:
 +                        line = self.single_statement
 +                        self.stop = True
 +                    else:
 +                        line = self.get_input_line(self.prompt)
 +                    self.statement.write(line)
 +                    if self.onecmd(self.statement.getvalue()):
 +                        self.reset_statement()
 +                except EOFError:
 +                    self.handle_eof()
 +                except CQL_ERRORS, cqlerr:
 +                    self.printerr(str(cqlerr))
 +                except KeyboardInterrupt:
 +                    self.reset_statement()
 +                    print
 +
 +    def onecmd(self, statementtext):
 +        """
 +        Returns true if the statement is complete and was handled (meaning it
 +        can be reset).
 +        """
 +
 +        try:
 +            statements, endtoken_escaped = cqlruleset.cql_split_statements(statementtext)
 +        except pylexotron.LexingError, e:
 +            if self.show_line_nums:
 +                self.printerr('Invalid syntax at char %d' % (e.charnum,))
 +            else:
 +                self.printerr('Invalid syntax at line %d, char %d'
 +                              % (e.linenum, e.charnum))
 +            statementline = statementtext.split('\n')[e.linenum - 1]
 +            self.printerr('  %s' % statementline)
 +            self.printerr(' %s^' % (' ' * e.charnum))
 +            return True
 +
 +        while statements and not statements[-1]:
 +            statements = statements[:-1]
 +        if not statements:
 +            return True
 +        if endtoken_escaped or statements[-1][-1][0] != 'endtoken':
 +            self.set_continue_prompt()
 +            return
 +        for st in statements:
 +            try:
 +                self.handle_statement(st, statementtext)
 +            except Exception, e:
 +                if self.debug:
 +                    traceback.print_exc()
 +                else:
 +                    self.printerr(e)
 +        return True
 +
 +    def handle_eof(self):
 +        if self.tty:
 +            print
 +        statement = self.statement.getvalue()
 +        if statement.strip():
 +            if not self.onecmd(statement):
 +                self.printerr('Incomplete statement at end of file')
 +        self.do_exit()
 +
 +    def handle_statement(self, tokens, srcstr):
 +        # Concat multi-line statements and insert into history
 +        if readline is not None:
 +            nl_count = srcstr.count("\n")
 +
 +            new_hist = srcstr.replace("\n", " ").rstrip()
 +
 +            if nl_count > 1 and self.last_hist != new_hist:
 +                readline.add_history(new_hist.encode(self.encoding))
 +
 +            self.last_hist = new_hist
 +        cmdword = tokens[0][1]
 +        if cmdword == '?':
 +            cmdword = 'help'
 +        custom_handler = getattr(self, 'do_' + cmdword.lower(), None)
 +        if custom_handler:
 +            parsed = cqlruleset.cql_whole_parse_tokens(tokens, srcstr=srcstr,
 +                                                       startsymbol='cqlshCommand')
 +            if parsed and not parsed.remainder:
 +                # successful complete parse
 +                return custom_handler(parsed)
 +            else:
 +                return self.handle_parse_error(cmdword, tokens, parsed, srcstr)
 +        return self.perform_statement(cqlruleset.cql_extract_orig(tokens, srcstr))
 +
 +    def handle_parse_error(self, cmdword, tokens, parsed, srcstr):
 +        if cmdword.lower() in ('select', 'insert', 'update', 'delete', 'truncate',
 +                               'create', 'drop', 'alter', 'grant', 'revoke',
 +                               'batch', 'list'):
 +            # hey, maybe they know about some new syntax we don't. type
 +            # assumptions won't work, but maybe the query will.
 +            return self.perform_statement(cqlruleset.cql_extract_orig(tokens, srcstr))
 +        if parsed:
 +            self.printerr('Improper %s command (problem at %r).' % (cmdword, parsed.remainder[0]))
 +        else:
 +            self.printerr('Improper %s command.' % cmdword)
 +
 +    def do_use(self, parsed):
 +        ksname = parsed.get_binding('ksname')
 +        success, _ = self.perform_simple_statement(SimpleStatement(parsed.extract_orig()))
 +        if success:
 +            if ksname[0] == '"' and ksname[-1] == '"':
 +                self.current_keyspace = self.cql_unprotect_name(ksname)
 +            else:
 +                self.current_keyspace = ksname.lower()
 +
 +    def do_select(self, parsed):
 +        tracing_was_enabled = self.tracing_enabled
 +        ksname = parsed.get_binding('ksname')
 +        stop_tracing = ksname == 'system_traces' or (ksname is None and self.current_keyspace == 'system_traces')
 +        self.tracing_enabled = self.tracing_enabled and not stop_tracing
 +        statement = parsed.extract_orig()
 +        self.perform_statement(statement)
 +        self.tracing_enabled = tracing_was_enabled
 +
 +    def perform_statement(self, statement):
 +        stmt = SimpleStatement(statement, consistency_level=self.consistency_level, serial_consistency_level=self.serial_consistency_level, fetch_size=self.page_size if self.use_paging else None)
 +        success, future = self.perform_simple_statement(stmt)
 +
 +        if future:
 +            if future.warnings:
 +                self.print_warnings(future.warnings)
 +
 +            if self.tracing_enabled:
 +                try:
 +                    for trace in future.get_all_query_traces(self.max_trace_wait):
 +                        print_trace(self, trace)
 +                except TraceUnavailable:
 +                    msg = "Statement trace did not complete within %d seconds; trace data may be incomplete." % (self.session.max_trace_wait,)
 +                    self.writeresult(msg, color=RED)
 +                    for trace_id in future.get_query_trace_ids():
 +                        self.show_session(trace_id, partial_session=True)
 +                except Exception, err:
 +                    self.printerr("Unable to fetch query trace: %s" % (str(err),))
 +
 +        return success
 +
 +    def parse_for_table_meta(self, query_string):
 +        try:
 +            parsed = cqlruleset.cql_parse(query_string)[1]
 +        except IndexError:
 +            return None
 +        ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +        cf = self.cql_unprotect_name(parsed.get_binding('cfname'))
 +        return self.get_table_meta(ks, cf)
 +
 +    def perform_simple_statement(self, statement):
 +        if not statement:
 +            return False, None
 +
 +        while True:
 +            try:
 +                future = self.session.execute_async(statement, trace=self.tracing_enabled)
 +                result = future.result()
 +                break
 +            except cassandra.OperationTimedOut, err:
 +                self.refresh_schema_metadata_best_effort()
 +                self.printerr(str(err.__class__.__name__) + ": " + str(err))
 +                return False, None
 +            except CQL_ERRORS, err:
 +                self.printerr(str(err.__class__.__name__) + ": " + str(err))
 +                return False, None
 +            except Exception, err:
 +                import traceback
 +                self.printerr(traceback.format_exc())
 +                return False, None
 +
 +        if statement.query_string[:6].lower() == 'select':
 +            self.print_result(result, self.parse_for_table_meta(statement.query_string))
 +        elif statement.query_string.lower().startswith("list users") or statement.query_string.lower().startswith("list roles"):
 +            self.print_result(result, self.get_table_meta('system_auth', 'roles'))
 +        elif statement.query_string.lower().startswith("list"):
 +            self.print_result(result, self.get_table_meta('system_auth', 'role_permissions'))
 +        elif result:
 +            # CAS INSERT/UPDATE
 +            self.writeresult("")
 +            self.print_static_result(result.column_names, list(result), self.parse_for_table_meta(statement.query_string))
 +        self.flush_output()
 +        return True, future
 +
 +    def print_result(self, result, table_meta):
 +        self.decoding_errors = []
 +
 +        self.writeresult("")
 +        if result.has_more_pages and self.tty:
 +            num_rows = 0
 +            while True:
 +                page = result.current_rows
 +                if page:
 +                    num_rows += len(page)
 +                    self.print_static_result(result.column_names, page, table_meta)
 +                if result.has_more_pages:
 +                    raw_input("---MORE---")
 +                    result.fetch_next_page()
 +                else:
 +                    break
 +        else:
 +            rows = list(result)
 +            num_rows = len(rows)
 +            self.print_static_result(result.column_names, rows, table_meta)
 +        self.writeresult("(%d rows)" % num_rows)
 +
 +        if self.decoding_errors:
 +            for err in self.decoding_errors[:2]:
 +                self.writeresult(err.message(), color=RED)
 +            if len(self.decoding_errors) > 2:
 +                self.writeresult('%d more decoding errors suppressed.'
 +                                 % (len(self.decoding_errors) - 2), color=RED)
 +
 +    def print_static_result(self, column_names, rows, table_meta):
 +        if not column_names and not table_meta:
 +            return
 +        column_names = column_names or table_meta.columns.keys()
 +        formatted_names = [self.myformat_colname(name, table_meta) for name in column_names]
 +        formatted_values = [map(self.myformat_value, row.values()) for row in rows]
 +
 +        if self.expand_enabled:
 +            self.print_formatted_result_vertically(formatted_names, formatted_values)
 +        else:
 +            self.print_formatted_result(formatted_names, formatted_values)
 +
 +    def print_formatted_result(self, formatted_names, formatted_values):
 +        # determine column widths
 +        widths = [n.displaywidth for n in formatted_names]
 +        if formatted_values is not None:
 +            for fmtrow in formatted_values:
 +                for num, col in enumerate(fmtrow):
 +                    widths[num] = max(widths[num], col.displaywidth)
 +
 +        # print header
 +        header = ' | '.join(hdr.ljust(w, color=self.color) for (hdr, w) in zip(formatted_names, widths))
 +        self.writeresult(' ' + header.rstrip())
 +        self.writeresult('-%s-' % '-+-'.join('-' * w for w in widths))
 +
 +        # stop if there are no rows
 +        if formatted_values is None:
 +            self.writeresult("")
 +            return
 +
 +        # print row data
 +        for row in formatted_values:
 +            line = ' | '.join(col.rjust(w, color=self.color) for (col, w) in zip(row, widths))
 +            self.writeresult(' ' + line)
 +
 +        self.writeresult("")
 +
 +    def print_formatted_result_vertically(self, formatted_names, formatted_values):
 +        max_col_width = max([n.displaywidth for n in formatted_names])
 +        max_val_width = max([n.displaywidth for row in formatted_values for n in row])
 +
 +        # for each row returned, list all the column-value pairs
 +        for row_id, row in enumerate(formatted_values):
 +            self.writeresult("@ Row %d" % (row_id + 1))
 +            self.writeresult('-%s-' % '-+-'.join(['-' * max_col_width, '-' * max_val_width]))
 +            for field_id, field in enumerate(row):
 +                column = formatted_names[field_id].ljust(max_col_width, color=self.color)
 +                value = field.ljust(field.displaywidth, color=self.color)
 +                self.writeresult(' ' + " | ".join([column, value]))
 +            self.writeresult('')
 +
 +    def print_warnings(self, warnings):
 +        if warnings is None or len(warnings) == 0:
 +            return
 +
 +        self.writeresult('')
 +        self.writeresult('Warnings :')
 +        for warning in warnings:
 +            self.writeresult(warning)
 +            self.writeresult('')
 +
 +    def emptyline(self):
 +        pass
 +
 +    def parseline(self, line):
 +        # this shouldn't be needed
 +        raise NotImplementedError
 +
 +    def complete(self, text, state):
 +        if readline is None:
 +            return
 +        if state == 0:
 +            try:
 +                self.completion_matches = self.find_completions(text)
 +            except Exception:
 +                if debug_completion:
 +                    import traceback
 +                    traceback.print_exc()
 +                else:
 +                    raise
 +        try:
 +            return self.completion_matches[state]
 +        except IndexError:
 +            return None
 +
 +    def find_completions(self, text):
 +        curline = readline.get_line_buffer()
 +        prevlines = self.statement.getvalue()
 +        wholestmt = prevlines + curline
 +        begidx = readline.get_begidx() + len(prevlines)
 +        stuff_to_complete = wholestmt[:begidx]
 +        return cqlruleset.cql_complete(stuff_to_complete, text, cassandra_conn=self,
 +                                       debug=debug_completion, startsymbol='cqlshCommand')
 +
 +    def set_prompt(self, prompt, prepend_user=False):
 +        if prepend_user and self.username:
 +            self.prompt = "%s@%s" % (self.username, prompt)
 +            return
 +        self.prompt = prompt
 +
 +    def cql_unprotect_name(self, namestr):
 +        if namestr is None:
 +            return
 +        return cqlruleset.dequote_name(namestr)
 +
 +    def cql_unprotect_value(self, valstr):
 +        if valstr is not None:
 +            return cqlruleset.dequote_value(valstr)
 +
 +    def print_recreate_keyspace(self, ksdef, out):
 +        out.write(ksdef.export_as_string())
 +        out.write("\n")
 +
 +    def print_recreate_columnfamily(self, ksname, cfname, out):
 +        """
 +        Output CQL commands which should be pasteable back into a CQL session
 +        to recreate the given table.
 +
 +        Writes output to the given out stream.
 +        """
 +        out.write(self.get_table_meta(ksname, cfname).export_as_string())
 +        out.write("\n")
 +
 +    def print_recreate_index(self, ksname, idxname, out):
 +        """
 +        Output CQL commands which should be pasteable back into a CQL session
 +        to recreate the given index.
 +
 +        Writes output to the given out stream.
 +        """
 +        out.write(self.get_index_meta(ksname, idxname).export_as_string())
 +        out.write("\n")
 +
 +    def print_recreate_object(self, ks, name, out):
 +        """
 +        Output CQL commands which should be pasteable back into a CQL session
 +        to recreate the given object (ks, table or index).
 +
 +        Writes output to the given out stream.
 +        """
 +        out.write(self.get_object_meta(ks, name).export_as_string())
 +        out.write("\n")
 +
 +    def describe_keyspaces(self):
 +        print
 +        cmd.Cmd.columnize(self, protect_names(self.get_keyspace_names()))
 +        print
 +
 +    def describe_keyspace(self, ksname):
 +        print
 +        self.print_recreate_keyspace(self.get_keyspace_meta(ksname), sys.stdout)
 +        print
 +
 +    def describe_columnfamily(self, ksname, cfname):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        if ksname is None:
 +            raise NoKeyspaceError("No keyspace specified and no current keyspace")
 +        print
 +        self.print_recreate_columnfamily(ksname, cfname, sys.stdout)
 +        print
 +
 +    def describe_index(self, ksname, idxname):
 +        print
 +        self.print_recreate_index(ksname, idxname, sys.stdout)
 +        print
 +
 +    def describe_object(self, ks, name):
 +        print
 +        self.print_recreate_object(ks, name, sys.stdout)
 +        print
 +
 +    def describe_columnfamilies(self, ksname):
 +        print
 +        if ksname is None:
 +            for k in self.get_keyspaces():
 +                name = protect_name(k.name)
 +                print 'Keyspace %s' % (name,)
 +                print '---------%s' % ('-' * len(name))
 +                cmd.Cmd.columnize(self, protect_names(self.get_columnfamily_names(k.name)))
 +                print
 +        else:
 +            cmd.Cmd.columnize(self, protect_names(self.get_columnfamily_names(ksname)))
 +            print
 +
 +    def describe_functions(self, ksname):
 +        print
 +        if ksname is None:
 +            for ksmeta in self.get_keyspaces():
 +                name = protect_name(ksmeta.name)
 +                print 'Keyspace %s' % (name,)
 +                print '---------%s' % ('-' * len(name))
 +                cmd.Cmd.columnize(self, protect_names(ksmeta.functions.keys()))
 +                print
 +        else:
 +            ksmeta = self.get_keyspace_meta(ksname)
 +            cmd.Cmd.columnize(self, protect_names(ksmeta.functions.keys()))
 +            print
 +
 +    def describe_function(self, ksname, functionname):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        if ksname is None:
 +            raise NoKeyspaceError("No keyspace specified and no current keyspace")
 +        print
 +        ksmeta = self.get_keyspace_meta(ksname)
 +        functions = filter(lambda f: f.name == functionname, ksmeta.functions.values())
 +        if len(functions) == 0:
 +            raise FunctionNotFound("User defined function %r not found" % functionname)
 +        print "\n\n".join(func.as_cql_query(formatted=True) for func in functions)
 +        print
 +
 +    def describe_aggregates(self, ksname):
 +        print
 +        if ksname is None:
 +            for ksmeta in self.get_keyspaces():
 +                name = protect_name(ksmeta.name)
 +                print 'Keyspace %s' % (name,)
 +                print '---------%s' % ('-' * len(name))
 +                cmd.Cmd.columnize(self, protect_names(ksmeta.aggregates.keys()))
 +                print
 +        else:
 +            ksmeta = self.get_keyspace_meta(ksname)
 +            cmd.Cmd.columnize(self, protect_names(ksmeta.aggregates.keys()))
 +            print
 +
 +    def describe_aggregate(self, ksname, aggregatename):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        if ksname is None:
 +            raise NoKeyspaceError("No keyspace specified and no current keyspace")
 +        print
 +        ksmeta = self.get_keyspace_meta(ksname)
 +        aggregates = filter(lambda f: f.name == aggregatename, ksmeta.aggregates.values())
 +        if len(aggregates) == 0:
 +            raise FunctionNotFound("User defined aggregate %r not found" % aggregatename)
 +        print "\n\n".join(aggr.as_cql_query(formatted=True) for aggr in aggregates)
 +        print
 +
 +    def describe_usertypes(self, ksname):
 +        print
 +        if ksname is None:
 +            for ksmeta in self.get_keyspaces():
 +                name = protect_name(ksmeta.name)
 +                print 'Keyspace %s' % (name,)
 +                print '---------%s' % ('-' * len(name))
 +                cmd.Cmd.columnize(self, protect_names(ksmeta.user_types.keys()))
 +                print
 +        else:
 +            ksmeta = self.get_keyspace_meta(ksname)
 +            cmd.Cmd.columnize(self, protect_names(ksmeta.user_types.keys()))
 +            print
 +
 +    def describe_usertype(self, ksname, typename):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        if ksname is None:
 +            raise NoKeyspaceError("No keyspace specified and no current keyspace")
 +        print
 +        ksmeta = self.get_keyspace_meta(ksname)
 +        try:
 +            usertype = ksmeta.user_types[typename]
 +        except KeyError:
 +            raise UserTypeNotFound("User type %r not found" % typename)
 +        print usertype.as_cql_query(formatted=True)
 +        print
 +
 +    def describe_cluster(self):
 +        print '\nCluster: %s' % self.get_cluster_name()
 +        p = trim_if_present(self.get_partitioner(), 'org.apache.cassandra.dht.')
 +        print 'Partitioner: %s\n' % p
 +        # TODO: snitch?
 +        # snitch = trim_if_present(self.get_snitch(), 'org.apache.cassandra.locator.')
 +        # print 'Snitch: %s\n' % snitch
 +        if self.current_keyspace is not None and self.current_keyspace != 'system':
 +            print "Range ownership:"
 +            ring = self.get_ring(self.current_keyspace)
 +            for entry in ring.items():
 +                print ' %39s  [%s]' % (str(entry[0].value), ', '.join([host.address for host in entry[1]]))
 +            print
 +
 +    def describe_schema(self, include_system=False):
 +        print
 +        for k in self.get_keyspaces():
 +            if include_system or k.name not in cql3handling.SYSTEM_KEYSPACES:
 +                self.print_recreate_keyspace(k, sys.stdout)
 +                print
 +
 +    def do_describe(self, parsed):
 +        """
 +        DESCRIBE [cqlsh only]
 +
 +        (DESC may be used as a shorthand.)
 +
 +          Outputs information about the connected Cassandra cluster, or about
 +          the data objects stored in the cluster. Use in one of the following ways:
 +
 +        DESCRIBE KEYSPACES
 +
 +          Output the names of all keyspaces.
 +
 +        DESCRIBE KEYSPACE [<keyspacename>]
 +
 +          Output CQL commands that could be used to recreate the given keyspace,
 +          and the objects in it (such as tables, types, functions, etc.).
 +          In some cases, as the CQL interface matures, there will be some metadata
 +          about a keyspace that is not representable with CQL. That metadata will not be shown.
 +
 +          The '<keyspacename>' argument may be omitted, in which case the current
 +          keyspace will be described.
 +
 +        DESCRIBE TABLES
 +
 +          Output the names of all tables in the current keyspace, or in all
 +          keyspaces if there is no current keyspace.
 +
 +        DESCRIBE TABLE [<keyspace>.]<tablename>
 +
 +          Output CQL commands that could be used to recreate the given table.
 +          In some cases, as above, there may be table metadata which is not
 +          representable and which will not be shown.
 +
 +        DESCRIBE INDEX <indexname>
 +
 +          Output the CQL command that could be used to recreate the given index.
 +          In some cases, there may be index metadata which is not representable
 +          and which will not be shown.
 +
 +        DESCRIBE CLUSTER
 +
 +          Output information about the connected Cassandra cluster, such as the
 +          cluster name, and the partitioner and snitch in use. When you are
 +          connected to a non-system keyspace, also shows endpoint-range
 +          ownership information for the Cassandra ring.
 +
 +        DESCRIBE [FULL] SCHEMA
 +
 +          Output CQL commands that could be used to recreate the entire (non-system) schema.
 +          Works as though "DESCRIBE KEYSPACE k" was invoked for each non-system keyspace
 +          k. Use DESCRIBE FULL SCHEMA to include the system keyspaces.
 +
 +        DESCRIBE TYPES
 +
 +          Output the names of all user-defined-types in the current keyspace, or in all
 +          keyspaces if there is no current keyspace.
 +
 +        DESCRIBE TYPE [<keyspace>.]<type>
 +
 +          Output the CQL command that could be used to recreate the given user-defined-type.
 +
 +        DESCRIBE FUNCTIONS
 +
 +          Output the names of all user-defined-functions in the current keyspace, or in all
 +          keyspaces if there is no current keyspace.
 +
 +        DESCRIBE FUNCTION [<keyspace>.]<function>
 +
 +          Output the CQL command that could be used to recreate the given user-defined-function.
 +
 +        DESCRIBE AGGREGATES
 +
 +          Output the names of all user-defined-aggregates in the current keyspace, or in all
 +          keyspaces if there is no current keyspace.
 +
 +        DESCRIBE AGGREGATE [<keyspace>.]<aggregate>
 +
 +          Output the CQL command that could be used to recreate the given user-defined-aggregate.
 +
 +        DESCRIBE <objname>
 +
 +          Output CQL commands that could be used to recreate the entire object schema,
 +          where object can be either a keyspace or a table or an index (in this order).
 +  """
 +        what = parsed.matched[1][1].lower()
 +        if what == 'functions':
 +            self.describe_functions(self.current_keyspace)
 +        elif what == 'function':
 +            ksname = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            functionname = self.cql_unprotect_name(parsed.get_binding('udfname'))
 +            self.describe_function(ksname, functionname)
 +        elif what == 'aggregates':
 +            self.describe_aggregates(self.current_keyspace)
 +        elif what == 'aggregate':
 +            ksname = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            aggregatename = self.cql_unprotect_name(parsed.get_binding('udaname'))
 +            self.describe_aggregate(ksname, aggregatename)
 +        elif what == 'keyspaces':
 +            self.describe_keyspaces()
 +        elif what == 'keyspace':
 +            ksname = self.cql_unprotect_name(parsed.get_binding('ksname', ''))
 +            if not ksname:
 +                ksname = self.current_keyspace
 +                if ksname is None:
 +                    self.printerr('Not in any keyspace.')
 +                    return
 +            self.describe_keyspace(ksname)
 +        elif what in ('columnfamily', 'table'):
 +            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            cf = self.cql_unprotect_name(parsed.get_binding('cfname'))
 +            self.describe_columnfamily(ks, cf)
 +        elif what == 'index':
 +            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            idx = self.cql_unprotect_name(parsed.get_binding('idxname', None))
 +            self.describe_index(ks, idx)
 +        elif what in ('columnfamilies', 'tables'):
 +            self.describe_columnfamilies(self.current_keyspace)
 +        elif what == 'types':
 +            self.describe_usertypes(self.current_keyspace)
 +        elif what == 'type':
 +            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            ut = self.cql_unprotect_name(parsed.get_binding('utname'))
 +            self.describe_usertype(ks, ut)
 +        elif what == 'cluster':
 +            self.describe_cluster()
 +        elif what == 'schema':
 +            self.describe_schema(False)
 +        elif what == 'full' and parsed.matched[2][1].lower() == 'schema':
 +            self.describe_schema(True)
 +        elif what:
 +            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            name = self.cql_unprotect_name(parsed.get_binding('cfname'))
 +            if not name:
 +                name = self.cql_unprotect_name(parsed.get_binding('idxname', None))
 +            self.describe_object(ks, name)
 +    do_desc = do_describe
 +
 +    def do_copy(self, parsed):
 +        r"""
 +        COPY [cqlsh only]
 +
 +          COPY x FROM: Imports CSV data into a Cassandra table
 +          COPY x TO: Exports data from a Cassandra table in CSV format.
 +
 +        COPY <table_name> [ ( column [, ...] ) ]
 +             FROM ( '<file_pattern_1, file_pattern_2, ... file_pattern_n>' | STDIN )
 +             [ WITH <option>='value' [AND ...] ];
 +
 +        File patterns are either file names or valid python glob expressions, e.g. *.csv or folder/*.csv.
 +
 +        COPY <table_name> [ ( column [, ...] ) ]
 +             TO ( '<filename>' | STDOUT )
 +             [ WITH <option>='value' [AND ...] ];
 +
 +        Available common COPY options and defaults:
 +
 +          DELIMITER=','           - character that appears between records
 +          QUOTE='"'               - quoting character to be used to quote fields
 +          ESCAPE='\'              - character to appear before the QUOTE char when quoted
 +          HEADER=false            - whether to ignore the first line
 +          NULL=''                 - string that represents a null value
 +          DATETIMEFORMAT=         - timestamp strftime format
 +            '%Y-%m-%d %H:%M:%S%z'   defaults to time_format value in cqlshrc
 +          MAXATTEMPTS=5           - the maximum number of attempts per batch or range
 +          REPORTFREQUENCY=0.25    - the frequency with which we display status updates in seconds
 +          DECIMALSEP='.'          - the separator for decimal values
 +          THOUSANDSSEP=''         - the separator for thousands digit groups
 +          BOOLSTYLE='True,False'  - the representation for booleans, case insensitive, specify true followed by false,
 +                                    for example yes,no or 1,0
 +          NUMPROCESSES=n          - the number of worker processes, by default the number of cores minus one
 +                                    capped at 16
 +          CONFIGFILE=''           - a configuration file with the same format as .cqlshrc (see the Python ConfigParser
 +                                    documentation) where you can specify WITH options under the following optional
 +                                    sections: [copy], [copy-to], [copy-from], [copy:ks.table], [copy-to:ks.table],
 +                                    [copy-from:ks.table], where <ks> is your keyspace name and <table> is your table
 +                                    name. Options are read from these sections, in the order specified
 +                                    above, and command line options always override options in configuration files.
 +                                    Depending on the COPY direction, only the relevant copy-from or copy-to sections
 +                                    are used. If no configfile is specified then .cqlshrc is searched instead.
 +          RATEFILE=''             - an optional file where to print the output statistics
 +
 +        Available COPY FROM options and defaults:
 +
-           CHUNKSIZE=1000          - the size of chunks passed to worker processes
++          CHUNKSIZE=5000          - the size of chunks passed to worker processes
 +          INGESTRATE=100000       - an approximate ingest rate in rows per second
-           MINBATCHSIZE=2          - the minimum size of an import batch
++          MINBATCHSIZE=10         - the minimum size of an import batch
 +          MAXBATCHSIZE=20         - the maximum size of an import batch
 +          MAXROWS=-1              - the maximum number of rows, -1 means no maximum
 +          SKIPROWS=0              - the number of rows to skip
 +          SKIPCOLS=''             - a comma separated list of column names to skip
 +          MAXPARSEERRORS=-1       - the maximum global number of parsing errors, -1 means no maximum
 +          MAXINSERTERRORS=-1      - the maximum global number of insert errors, -1 means no maximum
 +          ERRFILE=''              - a file where to store all rows that could not be imported, by default this is
 +                                    import_ks_table.err where <ks> is your keyspace and <table> is your table name.
++          PREPAREDSTATEMENTS=True - whether to use prepared statements when importing, by default True. Set this to
++                                    False if you don't mind shifting data parsing to the cluster. The cluster will also
++                                    have to compile every batch statement. For large and oversized clusters
++                                    this will result in a faster import but for smaller clusters it may generate
++                                    timeouts.
 +
 +        Available COPY TO options and defaults:
 +
 +          ENCODING='utf8'          - encoding for CSV output
 +          PAGESIZE='1000'          - the page size for fetching results
 +          PAGETIMEOUT=10           - the page timeout in seconds for fetching results
 +          BEGINTOKEN=''            - the minimum token string to consider when exporting data
 +          ENDTOKEN=''              - the maximum token string to consider when exporting data
 +          MAXREQUESTS=6            - the maximum number of requests each worker process can work on in parallel
 +          MAXOUTPUTSIZE='-1'       - the maximum size of the output file measured in number of lines,
 +                                     beyond this maximum the output file will be split into segments,
 +                                     -1 means unlimited.
 +
 +        When entering CSV data on STDIN, you can use the sequence "\."
 +        on a line by itself to end the data input.
 +        """
 +
 +        ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +        if ks is None:
 +            ks = self.current_keyspace
 +            if ks is None:
 +                raise NoKeyspaceError("Not in any keyspace.")
 +        table = self.cql_unprotect_name(parsed.get_binding('cfname'))
 +        columns = parsed.get_binding('colnames', None)
 +        if columns is not None:
 +            columns = map(self.cql_unprotect_name, columns)
 +        else:
 +            # default to all known columns
 +            columns = self.get_column_names(ks, table)
 +
 +        fname = parsed.get_binding('fname', None)
 +        if fname is not None:
 +            fname = self.cql_unprotect_value(fname)
 +
 +        copyoptnames = map(str.lower, parsed.get_binding('optnames', ()))
 +        copyoptvals = map(self.cql_unprotect_value, parsed.get_binding('optvals', ()))
 +        opts = dict(zip(copyoptnames, copyoptvals))
 +
 +        direction = parsed.get_binding('dir').upper()
 +        if direction == 'FROM':
 +            task = ImportTask(self, ks, table, columns, fname, opts, DEFAULT_PROTOCOL_VERSION, CONFIG_FILE)
 +        elif direction == 'TO':
 +            task = ExportTask(self, ks, table, columns, fname, opts, DEFAULT_PROTOCOL_VERSION, CONFIG_FILE)
 +        else:
 +            raise SyntaxError("Unknown direction %s" % direction)
 +
 +        task.run()
 +
 +    def do_show(self, parsed):
 +        """
 +        SHOW [cqlsh only]
 +
 +          Displays information about the current cqlsh session. Can be called in
 +          the following ways:
 +
 +        SHOW VERSION
 +
 +          Shows the version and build of the connected Cassandra instance, as
 +          well as the versions of the CQL spec and the Thrift protocol that
 +          the connected Cassandra instance understands.
 +
 +        SHOW HOST
 +
 +          Shows where cqlsh is currently connected.
 +
 +        SHOW SESSION <sessionid>
 +
 +          Pretty-prints the requested tracing session.
 +        """
 +        showwhat = parsed.get_binding('what').lower()
 +        if showwhat == 'version':
 +            self.get_connection_versions()
 +            self.show_version()
 +        elif showwhat == 'host':
 +            self.show_host()
 +        elif showwhat.startswith('session'):
 +            session_id = parsed.get_binding('sessionid').lower()
 +            self.show_session(UUID(session_id))
 +        else:
 +            self.printerr('Wait, how do I show %r?' % (showwhat,))
 +
 +    def do_source(self, parsed):
 +        """
 +        SOURCE [cqlsh only]
 +
 +        Executes a file containing CQL statements. Gives the output for each
 +        statement in turn, if any, or any errors that occur along the way.
 +
 +        Errors do NOT abort execution of the CQL source file.
 +
 +        Usage:
 +
 +          SOURCE '<file>';
 +
 +        That is, the path to the file to be executed must be given inside a
 +        string literal. The path is interpreted relative to the current working
 +        directory. The tilde shorthand notation ('~/mydir') is supported for
 +        referring to $HOME.
 +
 +        See also the --file option to cqlsh.
 +        """
 +        fname = parsed.get_binding('fname')
 +        fname = os.path.expanduser(self.cql_unprotect_value(fname))
 +        try:
 +            encoding, bom_size = get_file_encoding_bomsize(fname)
 +            f = codecs.open(fname, 'r', encoding)
 +            f.seek(bom_size)
 +        except IOError, e:
 +            self.printerr('Could not open %r: %s' % (fname, e))
 +            return
 +        subshell = Shell(self.hostname, self.port,
 +                         color=self.color, encoding=self.encoding, stdin=f,
 +                         tty=False, use_conn=self.conn, cqlver=self.cql_version,
 +                         display_timestamp_format=self.display_timestamp_format,
 +                         display_date_format=self.display_date_format,
 +                         display_nanotime_format=self.display_nanotime_format,
 +                         display_float_precision=self.display_float_precision,
 +                         max_trace_wait=self.max_trace_wait)
 +        subshell.cmdloop()
 +        f.close()
 +
 +    def do_capture(self, parsed):
 +        """
 +        CAPTURE [cqlsh only]
 +
 +        Begins capturing command output and appending it to a specified file.
 +        Output will not be shown at the console while it is captured.
 +
 +        Usage:
 +
 +          CAPTURE '<file>';
 +          CAPTURE OFF;
 +          CAPTURE;
 +
 +        That is, the path to the file to be appended to must be given inside a
 +        string literal. The path is interpreted relative to the current working
 +        directory. The tilde shorthand notation ('~/mydir') is supported for
 +        referring to $HOME.
 +
 +        Only query result output is captured. Errors and output from cqlsh-only
 +        commands will still be shown in the cqlsh session.
 +
 +        To stop capturing output and show it in the cqlsh session again, use
 +        CAPTURE OFF.
 +
 +        To inspect the current capture configuration, use CAPTURE with no
 +        arguments.
 +        """
 +        fname = parsed.get_binding('fname')
 +        if fname is None:
 +            if self.shunted_query_out is not None:
 +                print "Currently capturing query output to %r." % (self.query_out.name,)
 +            else:
 +                print "Currently not capturing query output."
 +            return
 +
 +        if fname.upper() == 'OFF':
 +            if self.shunted_query_out is None:
 +                self.printerr('Not currently capturing output.')
 +                return
 +            self.query_out.close()
 +            self.query_out = self.shunted_query_out
 +            self.color = self.shunted_color
 +            self.shunted_query_out = None
 +            del self.shunted_color
 +            return
 +
 +        if self.shunted_query_out is not None:
 +            self.printerr('Already capturing output to %s. Use CAPTURE OFF'
 +                          ' to disable.' % (self.query_out.name,))
 +            return
 +
 +        fname = os.path.expanduser(self.cql_unprotect_value(fname))
 +        try:
 +            f = open(fname, 'a')
 +        except IOError, e:
 +            self.printerr('Could not open %r for append: %s' % (fname, e))
 +            return
 +        self.shunted_query_out = self.query_out
 +        self.shunted_color = self.color
 +        self.query_out = f
 +        self.color = False
 +        print 'Now capturing query output to %r.' % (fname,)
 +
 +    def do_tracing(self, parsed):
 +        """
 +        TRACING [cqlsh]
 +
 +          Enables or disables request tracing.
 +
 +        TRACING ON
 +
 +          Enables tracing for all further requests.
 +
 +        TRACING OFF
 +
 +          Disables tracing.
 +
 +        TRACING
 +
 +          TRACING with no arguments shows the current tracing status.
 +        """
 +        self.tracing_enabled = SwitchCommand("TRACING", "Tracing").execute(self.tracing_enabled, parsed, self.printerr)
 +
 +    def do_expand(self, parsed):
 +        """
 +        EXPAND [cqlsh]
 +
 +          Enables or disables expanded (vertical) output.
 +
 +        EXPAND ON
 +
 +          Enables expanded (vertical) output.
 +
 +        EXPAND OFF
 +
 +          Disables expanded (vertical) output.
 +
 +        EXPAND
 +
 +          EXPAND with no arguments shows the current value of expand setting.
 +        """
 +        self.expand_enabled = SwitchCommand("EXPAND", "Expanded output").execute(self.expand_enabled, parsed, self.printerr)
 +
 +    def do_consistency(self, parsed):
 +        """
 +        CONSISTENCY [cqlsh only]
 +
 +           Overrides default consistency level (default level is ONE).
 +
 +        CONSISTENCY <level>
 +
 +           Sets consistency level for future requests.
 +
 +           Valid consistency levels:
 +
 +           ANY, ONE, TWO, THREE, QUORUM, ALL, LOCAL_ONE, LOCAL_QUORUM, EACH_QUORUM, SERIAL and LOCAL_SERIAL.
 +
 +           SERIAL and LOCAL_SERIAL may be used only for SELECTs; will be rejected with updates.
 +
 +        CONSISTENCY
 +
 +           CONSISTENCY with no arguments shows the current consistency level.
 +        """
 +        level = parsed.get_binding('level')
 +        if level is None:
 +            print 'Current consistency level is %s.' % (cassandra.ConsistencyLevel.value_to_name[self.consistency_level])
 +            return
 +
 +        self.consistency_level = cassandra.ConsistencyLevel.name_to_value[level.upper()]
 +        print 'Consistency level set to %s.' % (level.upper(),)
 +
 +    def do_serial(self, parsed):
 +        """
 +        SERIAL CONSISTENCY [cqlsh only]
 +
 +           Overrides serial consistency level (default level is SERIAL).
 +
 +        SERIAL CONSISTENCY <level>
 +
 +           Sets consistency level for future conditional updates.
 +
 +           Valid consistency levels:
 +
 +           SERIAL, LOCAL_SERIAL.
 +
 +        SERIAL CONSISTENCY
 +
 +           SERIAL CONSISTENCY with no arguments shows the current consistency level.
 +        """
 +        level = parsed.get_binding('level')
 +        if level is None:
 +            print 'Current serial consistency level is %s.' % (cassandra.ConsistencyLevel.value_to_name[self.serial_consistency_level])
 +            return
 +
 +        self.serial_consistency_level = cassandra.ConsistencyLevel.name_to_value[level.upper()]
 +        print 'Serial consistency level set to %s.' % (level.upper(),)
 +
 +    def do_login(self, parsed):
 +        """
 +        LOGIN [cqlsh only]
 +
 +           Changes login information without requiring restart.
 +
 +        LOGIN <username> (<password>)
 +
 +           Login using the specified username. If password is specified, it will be used
 +           otherwise, you will be prompted to enter.
 +        """
 +        username = parsed.get_binding('username')
 +        password = parsed.get_binding('password')
 +        if password is None:
 +            password = getpass.getpass()
 +        else:
 +            password = password[1:-1]
 +
 +        auth_provider = PlainTextAuthProvider(username=username, password=password)
 +
 +        conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=self.conn.cql_version,
 +                       protocol_version=self.conn.protocol_version,
 +                       auth_provider=auth_provider,
 +                       ssl_options=self.conn.ssl_options,
 +                       load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]),
 +                       control_connection_timeout=self.conn.connect_timeout,
 +                       connect_timeout=self.conn.connect_timeout)
 +
 +        if self.current_keyspace:
 +            session = conn.connect(self.current_keyspace)
 +        else:
 +            session = conn.connect()
 +
 +        # Update after we've connected in case we fail to authenticate
 +        self.conn = conn
 +        self.auth_provider = auth_provider
 +        self.username = username
 +        self.session = session
 +
 +    def do_exit(self, parsed=None):
 +        """
 +        EXIT/QUIT [cqlsh only]
 +
 +        Exits cqlsh.
 +        """
 +        self.stop = True
 +        if self.owns_connection:
 +            self.conn.shutdown()
 +    do_quit = do_exit
 +
 +    def do_clear(self, parsed):
 +        """
 +        CLEAR/CLS [cqlsh only]
 +
 +        Clears the console.
 +        """
 +        import subprocess
 +        subprocess.call(['clear', 'cls'][is_win], shell=True)
 +    do_cls = do_clear
 +
 +    def do_debug(self, parsed):
 +        import pdb
 +        pdb.set_trace()
 +
 +    def get_help_topics(self):
 +        topics = [t[3:] for t in dir(self) if t.startswith('do_') and getattr(self, t, None).__doc__]
 +        for hide_from_help in ('quit',):
 +            topics.remove(hide_from_help)
 +        return topi

<TRUNCATED>

[12/23] cassandra git commit: Merge commit 'c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc' into cassandra-2.2

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b74ffeaf/pylib/cqlshlib/copyutil.py
----------------------------------------------------------------------
diff --cc pylib/cqlshlib/copyutil.py
index aeb2d0b,cd03765..2755dd5
--- a/pylib/cqlshlib/copyutil.py
+++ b/pylib/cqlshlib/copyutil.py
@@@ -45,9 -51,13 +51,13 @@@ from cassandra.util import Date, Tim
  
  from cql3handling import CqlRuleSet
  from displaying import NO_COLOR_MAP
 -from formatting import format_value_default, EMPTY, get_formatter
 +from formatting import format_value_default, DateTimeFormat, EMPTY, get_formatter
  from sslhandling import ssl_settings
  
+ PROFILE_ON = False
+ STRACE_ON = False
+ IS_LINUX = platform.system() == 'Linux'
+ 
  CopyOptions = namedtuple('CopyOptions', 'copy dialect unrecognized')
  
  
@@@ -164,13 -252,12 +252,13 @@@ class CopyTask(object)
          # in the page size or 10 seconds if pagesize is smaller
          copy_options['pagetimeout'] = int(opts.pop('pagetimeout', max(10, 10 * (copy_options['pagesize'] / 1000))))
          copy_options['maxattempts'] = int(opts.pop('maxattempts', 5))
 -        copy_options['dtformats'] = opts.pop('datetimeformat', shell.display_time_format)
 +        copy_options['dtformats'] = DateTimeFormat(opts.pop('datetimeformat', shell.display_timestamp_format),
 +                                                   shell.display_date_format, shell.display_nanotime_format)
          copy_options['float_precision'] = shell.display_float_precision
-         copy_options['chunksize'] = int(opts.pop('chunksize', 1000))
+         copy_options['chunksize'] = int(opts.pop('chunksize', 5000))
          copy_options['ingestrate'] = int(opts.pop('ingestrate', 100000))
          copy_options['maxbatchsize'] = int(opts.pop('maxbatchsize', 20))
-         copy_options['minbatchsize'] = int(opts.pop('minbatchsize', 2))
+         copy_options['minbatchsize'] = int(opts.pop('minbatchsize', 10))
          copy_options['reportfrequency'] = float(opts.pop('reportfrequency', 0.25))
          copy_options['consistencylevel'] = shell.consistency_level
          copy_options['decimalsep'] = opts.pop('decimalsep', '.')
@@@ -1392,9 -1593,37 +1594,37 @@@ class ImportConversion(object)
          self.primary_key_indexes = [self.columns.index(col.name) for col in self.table_meta.primary_key]
          self.partition_key_indexes = [self.columns.index(col.name) for col in self.table_meta.partition_key]
  
+         if statement is None:
+             self.use_prepared_statements = False
+             statement = self._get_primary_key_statement(parent, table_meta)
+         else:
+             self.use_prepared_statements = True
+ 
          self.proto_version = statement.protocol_version
-         self.cqltypes = dict([(c.name, c.type) for c in statement.column_metadata])
-         self.converters = dict([(c.name, self._get_converter(c.type)) for c in statement.column_metadata])
+ 
+         # the cql types and converters for the prepared statement, either the full statement or only the primary keys
+         self.cqltypes = [c.type for c in statement.column_metadata]
+         self.converters = [self._get_converter(c.type) for c in statement.column_metadata]
+ 
+         # the cql types for the entire statement, these are the same as the types above but
+         # only when using prepared statements
 -        self.coltypes = [table_meta.columns[name].typestring for name in parent.valid_columns]
++        self.coltypes = [table_meta.columns[name].cql_type for name in parent.valid_columns]
+         # these functions are used for non-prepared statements to protect values with quotes if required
+         self.protectors = [protect_value if t in ('ascii', 'text', 'timestamp', 'date', 'time', 'inet') else lambda v: v
+                            for t in self.coltypes]
+ 
+     @staticmethod
+     def _get_primary_key_statement(parent, table_meta):
+         """
+         We prepare a query statement to find out the types of the partition key columns so we can
+         route the update query to the correct replicas. As far as I understood this is the easiest
+         way to find out the types of the partition columns, we will never use this prepared statement
+         """
+         where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
+         select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(parent.ks),
+                                                          protect_name(parent.table),
+                                                          where_clause)
+         return parent.session.prepare(select_query)
  
      def _get_converter(self, cql_type):
          """
@@@ -1695,67 -2007,88 +2008,88 @@@ class ImportProcess(ChildProcess)
              self._session.cluster.shutdown()
          ChildProcess.close(self)
  
-     def run_counter(self, table_meta):
-         """
-         Main run method for tables that contain counter columns.
-         """
-         query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
- 
-         # We prepare a query statement to find out the types of the partition key columns so we can
-         # route the update query to the correct replicas. As far as I understood this is the easiest
-         # way to find out the types of the partition columns, we will never use this prepared statement
-         where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
-         select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(self.ks), protect_name(self.table), where_clause)
-         conv = ImportConversion(self, table_meta, self.session.prepare(select_query))
- 
-         while True:
-             batch = self.inmsg.get()
-             try:
-                 for b in self.split_batches(batch, conv):
-                     self.send_counter_batch(query, conv, b)
+     def make_params(self):
+         metadata = self.session.cluster.metadata
+         table_meta = metadata.keyspaces[self.ks].tables[self.table]
+ 
+         prepared_statement = None
 -        is_counter = ("counter" in [table_meta.columns[name].typestring for name in self.valid_columns])
++        is_counter = ("counter" in [table_meta.columns[name].cql_type for name in self.valid_columns])
+         if is_counter:
+             query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
+             make_statement = self.wrap_make_statement(self.make_counter_batch_statement)
+         elif self.use_prepared_statements:
+             query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
+                                                             protect_name(self.table),
+                                                             ', '.join(protect_names(self.valid_columns),),
+                                                             ', '.join(['?' for _ in self.valid_columns]))
+ 
+             query = self.session.prepare(query)
+             query.consistency_level = self.consistency_level
+             prepared_statement = query
+             make_statement = self.wrap_make_statement(self.make_prepared_batch_statement)
+         else:
+             query = 'INSERT INTO %s.%s (%s) VALUES (%%s)' % (protect_name(self.ks),
+                                                              protect_name(self.table),
+                                                              ', '.join(protect_names(self.valid_columns),))
+             make_statement = self.wrap_make_statement(self.make_non_prepared_batch_statement)
  
-             except Exception, exc:
-                 self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                 if self.debug:
-                     traceback.print_exc(exc)
+         conv = ImportConversion(self, table_meta, prepared_statement)
+         tm = TokenMap(self.ks, self.hostname, self.local_dc, self.session)
+         return query, conv, tm, make_statement
  
-     def run_normal(self, table_meta):
+     def inner_run(self, query, conv, tm, make_statement):
          """
-         Main run method for normal tables, i.e. tables that do not contain counter columns.
+         Main run method. Note that we bind self methods that are called inside loops
+         for performance reasons.
          """
-         query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
-                                                         protect_name(self.table),
-                                                         ', '.join(protect_names(self.valid_columns),),
-                                                         ', '.join(['?' for _ in self.valid_columns]))
+         self.query = query
+         self.conv = conv
+         self.make_statement = make_statement
  
-         query_statement = self.session.prepare(query)
-         query_statement.consistency_level = self.consistency_level
-         conv = ImportConversion(self, table_meta, query_statement)
+         convert_rows = self.convert_rows
+         split_into_batches = self.split_into_batches
+         result_callback = self.result_callback
+         err_callback = self.err_callback
+         session = self.session
  
          while True:
-             batch = self.inmsg.get()
+             chunk = self.inmsg.recv()
+             if chunk is None:
+                 break
+ 
              try:
-                 for b in self.split_batches(batch, conv):
-                     self.send_normal_batch(conv, query_statement, b)
+                 chunk['rows'] = convert_rows(conv, chunk)
+                 for replicas, batch in split_into_batches(chunk, conv, tm):
+                     statement = make_statement(query, conv, chunk, batch, replicas)
+                     future = session.execute_async(statement)
+                     future.add_callbacks(callback=result_callback, callback_args=(batch, chunk),
+                                          errback=err_callback, errback_args=(batch, chunk, replicas))
  
              except Exception, exc:
-                 self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                 if self.debug:
-                     traceback.print_exc(exc)
+                 self.report_error(exc, chunk, chunk['rows'])
  
-     def send_counter_batch(self, query_text, conv, batch):
-         if self.test_failures and self.maybe_inject_failures(batch):
-             return
+     def wrap_make_statement(self, inner_make_statement):
+         def make_statement(query, conv, chunk, batch, replicas):
+             try:
+                 return inner_make_statement(query, conv, batch, replicas)
+             except Exception, exc:
+                 print "Failed to make batch statement: {}".format(exc)
+                 self.report_error(exc, chunk, batch['rows'])
+                 return None
  
-         error_rows = []
-         batch_statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+         def make_statement_with_failures(query, conv, chunk, batch, replicas):
+             failed_batch = self.maybe_inject_failures(batch)
+             if failed_batch:
+                 return failed_batch
+             return make_statement(query, conv, chunk, batch, replicas)
  
-         for r in batch['rows']:
-             row = self.filter_row_values(r)
-             if len(row) != len(self.valid_columns):
-                 error_rows.append(row)
-                 continue
+         return make_statement_with_failures if self.test_failures else make_statement
  
+     def make_counter_batch_statement(self, query, conv, batch, replicas):
+         statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+         statement.replicas = replicas
+         statement.keyspace = self.ks
+         for row in batch['rows']:
              where_clause = []
              set_clause = []
              for i, value in enumerate(row):


[07/23] cassandra git commit: Merge commit 'c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc' into cassandra-2.2

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b74ffeaf/bin/cqlsh.py
----------------------------------------------------------------------
diff --cc bin/cqlsh.py
index b3ec1ac,0000000..9082d72
mode 100644,000000..100644
--- a/bin/cqlsh.py
+++ b/bin/cqlsh.py
@@@ -1,2546 -1,0 +1,2566 @@@
 +#!/bin/sh
 +# -*- mode: Python -*-
 +
 +# Licensed to the Apache Software Foundation (ASF) under one
 +# or more contributor license agreements.  See the NOTICE file
 +# distributed with this work for additional information
 +# regarding copyright ownership.  The ASF licenses this file
 +# to you under the Apache License, Version 2.0 (the
 +# "License"); you may not use this file except in compliance
 +# with the License.  You may obtain a copy of the License at
 +#
 +#     http://www.apache.org/licenses/LICENSE-2.0
 +#
 +# Unless required by applicable law or agreed to in writing, software
 +# distributed under the License is distributed on an "AS IS" BASIS,
 +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 +# See the License for the specific language governing permissions and
 +# limitations under the License.
 +
 +""":"
 +# bash code here; finds a suitable python interpreter and execs this file.
 +# prefer unqualified "python" if suitable:
 +python -c 'import sys; sys.exit(not (0x020700b0 < sys.hexversion < 0x03000000))' 2>/dev/null \
 +    && exec python "$0" "$@"
 +for pyver in 2.7; do
 +    which python$pyver > /dev/null 2>&1 && exec python$pyver "$0" "$@"
 +done
 +echo "No appropriate python interpreter found." >&2
 +exit 1
 +":"""
 +
 +from __future__ import with_statement
 +
 +import cmd
 +import codecs
 +import ConfigParser
 +import csv
 +import getpass
 +import locale
 +import optparse
 +import os
 +import platform
 +import sys
 +import traceback
 +import warnings
 +import webbrowser
 +from StringIO import StringIO
 +from contextlib import contextmanager
 +from glob import glob
 +from uuid import UUID
 +
 +if sys.version_info[0] != 2 or sys.version_info[1] != 7:
 +    sys.exit("\nCQL Shell supports only Python 2.7\n")
 +
 +UTF8 = 'utf-8'
 +CP65001 = 'cp65001'  # Win utf-8 variant
 +
 +description = "CQL Shell for Apache Cassandra"
 +version = "5.0.1"
 +
 +readline = None
 +try:
 +    # check if tty first, cause readline doesn't check, and only cares
 +    # about $TERM. we don't want the funky escape code stuff to be
 +    # output if not a tty.
 +    if sys.stdin.isatty():
 +        import readline
 +except ImportError:
 +    pass
 +
 +CQL_LIB_PREFIX = 'cassandra-driver-internal-only-'
 +
 +CASSANDRA_PATH = os.path.join(os.path.dirname(os.path.realpath(__file__)), '..')
 +CASSANDRA_CQL_HTML_FALLBACK = 'https://cassandra.apache.org/doc/cql3/CQL-2.2.html'
 +
 +if os.path.exists(CASSANDRA_PATH + '/doc/cql3/CQL.html'):
 +    # default location of local CQL.html
 +    CASSANDRA_CQL_HTML = 'file://' + CASSANDRA_PATH + '/doc/cql3/CQL.html'
 +elif os.path.exists('/usr/share/doc/cassandra/CQL.html'):
 +    # fallback to package file
 +    CASSANDRA_CQL_HTML = 'file:///usr/share/doc/cassandra/CQL.html'
 +else:
 +    # fallback to online version
 +    CASSANDRA_CQL_HTML = CASSANDRA_CQL_HTML_FALLBACK
 +
 +# On Linux, the Python webbrowser module uses the 'xdg-open' executable
 +# to open a file/URL. But that only works, if the current session has been
 +# opened from _within_ a desktop environment. I.e. 'xdg-open' will fail,
 +# if the session's been opened via ssh to a remote box.
 +#
 +# Use 'python' to get some information about the detected browsers.
 +# >>> import webbrowser
 +# >>> webbrowser._tryorder
 +# >>> webbrowser._browser
 +#
 +if len(webbrowser._tryorder) == 0:
 +    CASSANDRA_CQL_HTML = CASSANDRA_CQL_HTML_FALLBACK
 +elif webbrowser._tryorder[0] == 'xdg-open' and os.environ.get('XDG_DATA_DIRS', '') == '':
 +    # only on Linux (some OS with xdg-open)
 +    webbrowser._tryorder.remove('xdg-open')
 +    webbrowser._tryorder.append('xdg-open')
 +
 +# use bundled libs for python-cql and thrift, if available. if there
 +# is a ../lib dir, use bundled libs there preferentially.
 +ZIPLIB_DIRS = [os.path.join(CASSANDRA_PATH, 'lib')]
 +myplatform = platform.system()
 +is_win = myplatform == 'Windows'
 +
 +# Workaround for supporting CP65001 encoding on python < 3.3 (https://bugs.python.org/issue13216)
 +if is_win and sys.version_info < (3, 3):
 +    codecs.register(lambda name: codecs.lookup(UTF8) if name == CP65001 else None)
 +
 +if myplatform == 'Linux':
 +    ZIPLIB_DIRS.append('/usr/share/cassandra/lib')
 +
 +if os.environ.get('CQLSH_NO_BUNDLED', ''):
 +    ZIPLIB_DIRS = ()
 +
 +
 +def find_zip(libprefix):
 +    for ziplibdir in ZIPLIB_DIRS:
 +        zips = glob(os.path.join(ziplibdir, libprefix + '*.zip'))
 +        if zips:
 +            return max(zips)   # probably the highest version, if multiple
 +
 +cql_zip = find_zip(CQL_LIB_PREFIX)
 +if cql_zip:
 +    ver = os.path.splitext(os.path.basename(cql_zip))[0][len(CQL_LIB_PREFIX):]
 +    sys.path.insert(0, os.path.join(cql_zip, 'cassandra-driver-' + ver))
 +
 +third_parties = ('futures-', 'six-')
 +
 +for lib in third_parties:
 +    lib_zip = find_zip(lib)
 +    if lib_zip:
 +        sys.path.insert(0, lib_zip)
 +
 +warnings.filterwarnings("ignore", r".*blist.*")
 +try:
 +    import cassandra
 +except ImportError, e:
 +    sys.exit("\nPython Cassandra driver not installed, or not on PYTHONPATH.\n"
 +             'You might try "pip install cassandra-driver".\n\n'
 +             'Python: %s\n'
 +             'Module load path: %r\n\n'
 +             'Error: %s\n' % (sys.executable, sys.path, e))
 +
 +from cassandra.auth import PlainTextAuthProvider
 +from cassandra.cluster import Cluster
 +from cassandra.metadata import (ColumnMetadata, KeyspaceMetadata,
 +                                TableMetadata, protect_name, protect_names)
 +from cassandra.policies import WhiteListRoundRobinPolicy
 +from cassandra.query import SimpleStatement, ordered_dict_factory, TraceUnavailable
 +
 +# cqlsh should run correctly when run out of a Cassandra source tree,
 +# out of an unpacked Cassandra tarball, and after a proper package install.
 +cqlshlibdir = os.path.join(CASSANDRA_PATH, 'pylib')
 +if os.path.isdir(cqlshlibdir):
 +    sys.path.insert(0, cqlshlibdir)
 +
 +from cqlshlib import cql3handling, cqlhandling, pylexotron, sslhandling
 +from cqlshlib.copyutil import ExportTask, ImportTask
 +from cqlshlib.displaying import (ANSI_RESET, BLUE, COLUMN_NAME_COLORS, CYAN,
 +                                 RED, FormattedValue, colorme)
 +from cqlshlib.formatting import (DEFAULT_DATE_FORMAT, DEFAULT_NANOTIME_FORMAT,
 +                                 DEFAULT_TIMESTAMP_FORMAT, DateTimeFormat,
 +                                 format_by_type, format_value_utype,
 +                                 formatter_for)
 +from cqlshlib.tracing import print_trace, print_trace_session
 +from cqlshlib.util import get_file_encoding_bomsize, trim_if_present
 +
 +DEFAULT_HOST = '127.0.0.1'
 +DEFAULT_PORT = 9042
 +DEFAULT_CQLVER = '3.3.1'
 +DEFAULT_PROTOCOL_VERSION = 4
 +DEFAULT_CONNECT_TIMEOUT_SECONDS = 5
 +DEFAULT_REQUEST_TIMEOUT_SECONDS = 10
 +
 +DEFAULT_FLOAT_PRECISION = 5
 +DEFAULT_MAX_TRACE_WAIT = 10
 +
 +if readline is not None and readline.__doc__ is not None and 'libedit' in readline.__doc__:
 +    DEFAULT_COMPLETEKEY = '\t'
 +else:
 +    DEFAULT_COMPLETEKEY = 'tab'
 +
 +cqldocs = None
 +cqlruleset = None
 +
 +epilog = """Connects to %(DEFAULT_HOST)s:%(DEFAULT_PORT)d by default. These
 +defaults can be changed by setting $CQLSH_HOST and/or $CQLSH_PORT. When a
 +host (and optional port number) are given on the command line, they take
 +precedence over any defaults.""" % globals()
 +
 +parser = optparse.OptionParser(description=description, epilog=epilog,
 +                               usage="Usage: %prog [options] [host [port]]",
 +                               version='cqlsh ' + version)
 +parser.add_option("-C", "--color", action='store_true', dest='color',
 +                  help='Always use color output')
 +parser.add_option("--no-color", action='store_false', dest='color',
 +                  help='Never use color output')
 +parser.add_option("--browser", dest='browser', help="""The browser to use to display CQL help, where BROWSER can be:
 +                                                    - one of the supported browsers in https://docs.python.org/2/library/webbrowser.html.
 +                                                    - browser path followed by %s, example: /usr/bin/google-chrome-stable %s""")
 +parser.add_option('--ssl', action='store_true', help='Use SSL', default=False)
 +parser.add_option("-u", "--username", help="Authenticate as user.")
 +parser.add_option("-p", "--password", help="Authenticate using password.")
 +parser.add_option('-k', '--keyspace', help='Authenticate to the given keyspace.')
 +parser.add_option("-f", "--file", help="Execute commands from FILE, then exit")
 +parser.add_option('--debug', action='store_true',
 +                  help='Show additional debugging information')
 +parser.add_option("--encoding", help="Specify a non-default encoding for output." +
 +                  " (Default: %s)" % (UTF8,))
 +parser.add_option("--cqlshrc", help="Specify an alternative cqlshrc file location.")
 +parser.add_option('--cqlversion', default=DEFAULT_CQLVER,
 +                  help='Specify a particular CQL version (default: %default).'
 +                       ' Examples: "3.0.3", "3.1.0"')
 +parser.add_option("-e", "--execute", help='Execute the statement and quit.')
 +parser.add_option("--connect-timeout", default=DEFAULT_CONNECT_TIMEOUT_SECONDS, dest='connect_timeout',
 +                  help='Specify the connection timeout in seconds (default: %default seconds).')
 +parser.add_option("--request-timeout", default=DEFAULT_REQUEST_TIMEOUT_SECONDS, dest='request_timeout',
 +                  help='Specify the default request timeout in seconds (default: %default seconds).')
 +parser.add_option("-t", "--tty", action='store_true', dest='tty',
 +                  help='Force tty mode (command prompt).')
 +
 +optvalues = optparse.Values()
 +(options, arguments) = parser.parse_args(sys.argv[1:], values=optvalues)
 +
 +# BEGIN history/config definition
 +HISTORY_DIR = os.path.expanduser(os.path.join('~', '.cassandra'))
 +
 +if hasattr(options, 'cqlshrc'):
 +    CONFIG_FILE = options.cqlshrc
 +    if not os.path.exists(CONFIG_FILE):
 +        print '\nWarning: Specified cqlshrc location `%s` does not exist.  Using `%s` instead.\n' % (CONFIG_FILE, HISTORY_DIR)
 +        CONFIG_FILE = os.path.join(HISTORY_DIR, 'cqlshrc')
 +else:
 +    CONFIG_FILE = os.path.join(HISTORY_DIR, 'cqlshrc')
 +
 +HISTORY = os.path.join(HISTORY_DIR, 'cqlsh_history')
 +if not os.path.exists(HISTORY_DIR):
 +    try:
 +        os.mkdir(HISTORY_DIR)
 +    except OSError:
 +        print '\nWarning: Cannot create directory at `%s`. Command history will not be saved.\n' % HISTORY_DIR
 +
 +OLD_CONFIG_FILE = os.path.expanduser(os.path.join('~', '.cqlshrc'))
 +if os.path.exists(OLD_CONFIG_FILE):
 +    if os.path.exists(CONFIG_FILE):
 +        print '\nWarning: cqlshrc config files were found at both the old location (%s) and \
 +                the new location (%s), the old config file will not be migrated to the new \
 +                location, and the new location will be used for now.  You should manually \
 +                consolidate the config files at the new location and remove the old file.' \
 +                % (OLD_CONFIG_FILE, CONFIG_FILE)
 +    else:
 +        os.rename(OLD_CONFIG_FILE, CONFIG_FILE)
 +OLD_HISTORY = os.path.expanduser(os.path.join('~', '.cqlsh_history'))
 +if os.path.exists(OLD_HISTORY):
 +    os.rename(OLD_HISTORY, HISTORY)
 +# END history/config definition
 +
 +CQL_ERRORS = (
 +    cassandra.AlreadyExists, cassandra.AuthenticationFailed, cassandra.InvalidRequest,
 +    cassandra.Timeout, cassandra.Unauthorized, cassandra.OperationTimedOut,
 +    cassandra.cluster.NoHostAvailable,
 +    cassandra.connection.ConnectionBusy, cassandra.connection.ProtocolError, cassandra.connection.ConnectionException,
 +    cassandra.protocol.ErrorMessage, cassandra.protocol.InternalError, cassandra.query.TraceUnavailable
 +)
 +
 +debug_completion = bool(os.environ.get('CQLSH_DEBUG_COMPLETION', '') == 'YES')
 +
 +# we want the cql parser to understand our cqlsh-specific commands too
 +my_commands_ending_with_newline = (
 +    'help',
 +    '?',
 +    'consistency',
 +    'serial',
 +    'describe',
 +    'desc',
 +    'show',
 +    'source',
 +    'capture',
 +    'login',
 +    'debug',
 +    'tracing',
 +    'expand',
 +    'paging',
 +    'exit',
 +    'quit',
 +    'clear',
 +    'cls'
 +)
 +
 +
 +cqlsh_syntax_completers = []
 +
 +
 +def cqlsh_syntax_completer(rulename, termname):
 +    def registrator(f):
 +        cqlsh_syntax_completers.append((rulename, termname, f))
 +        return f
 +    return registrator
 +
 +
 +cqlsh_extra_syntax_rules = r'''
 +<cqlshCommand> ::= <CQL_Statement>
 +                 | <specialCommand> ( ";" | "\n" )
 +                 ;
 +
 +<specialCommand> ::= <describeCommand>
 +                   | <consistencyCommand>
 +                   | <serialConsistencyCommand>
 +                   | <showCommand>
 +                   | <sourceCommand>
 +                   | <captureCommand>
 +                   | <copyCommand>
 +                   | <loginCommand>
 +                   | <debugCommand>
 +                   | <helpCommand>
 +                   | <tracingCommand>
 +                   | <expandCommand>
 +                   | <exitCommand>
 +                   | <pagingCommand>
 +                   | <clearCommand>
 +                   ;
 +
 +<describeCommand> ::= ( "DESCRIBE" | "DESC" )
 +                                  ( "FUNCTIONS"
 +                                  | "FUNCTION" udf=<anyFunctionName>
 +                                  | "AGGREGATES"
 +                                  | "AGGREGATE" uda=<userAggregateName>
 +                                  | "KEYSPACES"
 +                                  | "KEYSPACE" ksname=<keyspaceName>?
 +                                  | ( "COLUMNFAMILY" | "TABLE" ) cf=<columnFamilyName>
 +                                  | "INDEX" idx=<indexName>
 +                                  | ( "COLUMNFAMILIES" | "TABLES" )
 +                                  | "FULL"? "SCHEMA"
 +                                  | "CLUSTER"
 +                                  | "TYPES"
 +                                  | "TYPE" ut=<userTypeName>
 +                                  | (ksname=<keyspaceName> | cf=<columnFamilyName> | idx=<indexName>))
 +                    ;
 +
 +<consistencyCommand> ::= "CONSISTENCY" ( level=<consistencyLevel> )?
 +                       ;
 +
 +<consistencyLevel> ::= "ANY"
 +                     | "ONE"
 +                     | "TWO"
 +                     | "THREE"
 +                     | "QUORUM"
 +                     | "ALL"
 +                     | "LOCAL_QUORUM"
 +                     | "EACH_QUORUM"
 +                     | "SERIAL"
 +                     | "LOCAL_SERIAL"
 +                     | "LOCAL_ONE"
 +                     ;
 +
 +<serialConsistencyCommand> ::= "SERIAL" "CONSISTENCY" ( level=<serialConsistencyLevel> )?
 +                             ;
 +
 +<serialConsistencyLevel> ::= "SERIAL"
 +                           | "LOCAL_SERIAL"
 +                           ;
 +
 +<showCommand> ::= "SHOW" what=( "VERSION" | "HOST" | "SESSION" sessionid=<uuid> )
 +                ;
 +
 +<sourceCommand> ::= "SOURCE" fname=<stringLiteral>
 +                  ;
 +
 +<captureCommand> ::= "CAPTURE" ( fname=( <stringLiteral> | "OFF" ) )?
 +                   ;
 +
 +<copyCommand> ::= "COPY" cf=<columnFamilyName>
 +                         ( "(" [colnames]=<colname> ( "," [colnames]=<colname> )* ")" )?
 +                         ( dir="FROM" ( fname=<stringLiteral> | "STDIN" )
 +                         | dir="TO"   ( fname=<stringLiteral> | "STDOUT" ) )
 +                         ( "WITH" <copyOption> ( "AND" <copyOption> )* )?
 +                ;
 +
 +<copyOption> ::= [optnames]=(<identifier>|<reserved_identifier>) "=" [optvals]=<copyOptionVal>
 +               ;
 +
 +<copyOptionVal> ::= <identifier>
 +                  | <reserved_identifier>
 +                  | <term>
 +                  ;
 +
 +# avoiding just "DEBUG" so that this rule doesn't get treated as a terminal
 +<debugCommand> ::= "DEBUG" "THINGS"?
 +                 ;
 +
 +<helpCommand> ::= ( "HELP" | "?" ) [topic]=( /[a-z_]*/ )*
 +                ;
 +
 +<tracingCommand> ::= "TRACING" ( switch=( "ON" | "OFF" ) )?
 +                   ;
 +
 +<expandCommand> ::= "EXPAND" ( switch=( "ON" | "OFF" ) )?
 +                   ;
 +
 +<pagingCommand> ::= "PAGING" ( switch=( "ON" | "OFF" | /[0-9]+/) )?
 +                  ;
 +
 +<loginCommand> ::= "LOGIN" username=<username> (password=<stringLiteral>)?
 +                 ;
 +
 +<exitCommand> ::= "exit" | "quit"
 +                ;
 +
 +<clearCommand> ::= "CLEAR" | "CLS"
 +                 ;
 +
 +<qmark> ::= "?" ;
 +'''
 +
 +
 +@cqlsh_syntax_completer('helpCommand', 'topic')
 +def complete_help(ctxt, cqlsh):
 +    return sorted([t.upper() for t in cqldocs.get_help_topics() + cqlsh.get_help_topics()])
 +
 +
 +def complete_source_quoted_filename(ctxt, cqlsh):
 +    partial_path = ctxt.get_binding('partial', '')
 +    head, tail = os.path.split(partial_path)
 +    exhead = os.path.expanduser(head)
 +    try:
 +        contents = os.listdir(exhead or '.')
 +    except OSError:
 +        return ()
 +    matches = filter(lambda f: f.startswith(tail), contents)
 +    annotated = []
 +    for f in matches:
 +        match = os.path.join(head, f)
 +        if os.path.isdir(os.path.join(exhead, f)):
 +            match += '/'
 +        annotated.append(match)
 +    return annotated
 +
 +
 +cqlsh_syntax_completer('sourceCommand', 'fname')(complete_source_quoted_filename)
 +cqlsh_syntax_completer('captureCommand', 'fname')(complete_source_quoted_filename)
 +
 +
 +@cqlsh_syntax_completer('copyCommand', 'fname')
 +def copy_fname_completer(ctxt, cqlsh):
 +    lasttype = ctxt.get_binding('*LASTTYPE*')
 +    if lasttype == 'unclosedString':
 +        return complete_source_quoted_filename(ctxt, cqlsh)
 +    partial_path = ctxt.get_binding('partial')
 +    if partial_path == '':
 +        return ["'"]
 +    return ()
 +
 +
 +@cqlsh_syntax_completer('copyCommand', 'colnames')
 +def complete_copy_column_names(ctxt, cqlsh):
 +    existcols = map(cqlsh.cql_unprotect_name, ctxt.get_binding('colnames', ()))
 +    ks = cqlsh.cql_unprotect_name(ctxt.get_binding('ksname', None))
 +    cf = cqlsh.cql_unprotect_name(ctxt.get_binding('cfname'))
 +    colnames = cqlsh.get_column_names(ks, cf)
 +    if len(existcols) == 0:
 +        return [colnames[0]]
 +    return set(colnames[1:]) - set(existcols)
 +
 +
 +COPY_COMMON_OPTIONS = ['DELIMITER', 'QUOTE', 'ESCAPE', 'HEADER', 'NULL', 'DATETIMEFORMAT',
 +                       'MAXATTEMPTS', 'REPORTFREQUENCY', 'DECIMALSEP', 'THOUSANDSSEP', 'BOOLSTYLE',
 +                       'NUMPROCESSES', 'CONFIGFILE', 'RATEFILE']
 +COPY_FROM_OPTIONS = ['CHUNKSIZE', 'INGESTRATE', 'MAXBATCHSIZE', 'MINBATCHSIZE', 'MAXROWS',
-                      'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE']
++                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE', 'PREPAREDSTATEMENTS']
 +COPY_TO_OPTIONS = ['ENCODING', 'PAGESIZE', 'PAGETIMEOUT', 'BEGINTOKEN', 'ENDTOKEN', 'MAXOUTPUTSIZE', 'MAXREQUESTS']
 +
 +
 +@cqlsh_syntax_completer('copyOption', 'optnames')
 +def complete_copy_options(ctxt, cqlsh):
 +    optnames = map(str.upper, ctxt.get_binding('optnames', ()))
 +    direction = ctxt.get_binding('dir').upper()
 +    if direction == 'FROM':
 +        opts = set(COPY_COMMON_OPTIONS + COPY_FROM_OPTIONS) - set(optnames)
 +    elif direction == 'TO':
 +        opts = set(COPY_COMMON_OPTIONS + COPY_TO_OPTIONS) - set(optnames)
 +    return opts
 +
 +
 +@cqlsh_syntax_completer('copyOption', 'optvals')
 +def complete_copy_opt_values(ctxt, cqlsh):
 +    optnames = ctxt.get_binding('optnames', ())
 +    lastopt = optnames[-1].lower()
 +    if lastopt == 'header':
 +        return ['true', 'false']
 +    return [cqlhandling.Hint('<single_character_string>')]
 +
 +
 +class NoKeyspaceError(Exception):
 +    pass
 +
 +
 +class KeyspaceNotFound(Exception):
 +    pass
 +
 +
 +class ColumnFamilyNotFound(Exception):
 +    pass
 +
 +
 +class IndexNotFound(Exception):
 +    pass
 +
 +
 +class ObjectNotFound(Exception):
 +    pass
 +
 +
 +class VersionNotSupported(Exception):
 +    pass
 +
 +
 +class UserTypeNotFound(Exception):
 +    pass
 +
 +
 +class FunctionNotFound(Exception):
 +    pass
 +
 +
 +class AggregateNotFound(Exception):
 +    pass
 +
 +
 +class DecodeError(Exception):
 +    verb = 'decode'
 +
 +    def __init__(self, thebytes, err, colname=None):
 +        self.thebytes = thebytes
 +        self.err = err
 +        self.colname = colname
 +
 +    def __str__(self):
 +        return str(self.thebytes)
 +
 +    def message(self):
 +        what = 'value %r' % (self.thebytes,)
 +        if self.colname is not None:
 +            what = 'value %r (for column %r)' % (self.thebytes, self.colname)
 +        return 'Failed to %s %s : %s' \
 +               % (self.verb, what, self.err)
 +
 +    def __repr__(self):
 +        return '<%s %s>' % (self.__class__.__name__, self.message())
 +
 +
 +class FormatError(DecodeError):
 +    verb = 'format'
 +
 +
 +def full_cql_version(ver):
 +    while ver.count('.') < 2:
 +        ver += '.0'
 +    ver_parts = ver.split('-', 1) + ['']
 +    vertuple = tuple(map(int, ver_parts[0].split('.')) + [ver_parts[1]])
 +    return ver, vertuple
 +
 +
 +def format_value(val, output_encoding, addcolor=False, date_time_format=None,
 +                 float_precision=None, colormap=None, nullval=None):
 +    if isinstance(val, DecodeError):
 +        if addcolor:
 +            return colorme(repr(val.thebytes), colormap, 'error')
 +        else:
 +            return FormattedValue(repr(val.thebytes))
 +    return format_by_type(type(val), val, output_encoding, colormap=colormap,
 +                          addcolor=addcolor, nullval=nullval, date_time_format=date_time_format,
 +                          float_precision=float_precision)
 +
 +
 +def show_warning_without_quoting_line(message, category, filename, lineno, file=None, line=None):
 +    if file is None:
 +        file = sys.stderr
 +    try:
 +        file.write(warnings.formatwarning(message, category, filename, lineno, line=''))
 +    except IOError:
 +        pass
 +warnings.showwarning = show_warning_without_quoting_line
 +warnings.filterwarnings('always', category=cql3handling.UnexpectedTableStructure)
 +
 +
 +def insert_driver_hooks():
 +    extend_cql_deserialization()
 +    auto_format_udts()
 +
 +
 +def extend_cql_deserialization():
 +    """
-     The python driver returns BLOBs as string, but we expect them as bytearrays
++    The python driver returns BLOBs as string, but we expect them as bytearrays; therefore we change
++    the implementation of cassandra.cqltypes.BytesType.deserialize.
++
++    The deserializers package exists only when the driver has been compiled with cython extensions and
++    cassandra.deserializers.DesBytesType replaces cassandra.cqltypes.BytesType.deserialize.
++
++    DesBytesTypeByteArray is a fast deserializer that converts blobs into bytearrays but it was
++    only introduced recently (3.1.0). If it is available we use it, otherwise we remove
++    cassandra.deserializers.DesBytesType so that we fall back onto cassandra.cqltypes.BytesType.deserialize
++    just like in the case where no cython extensions are present.
 +    """
++    if hasattr(cassandra, 'deserializers'):
++        if hasattr(cassandra.deserializers, 'DesBytesTypeByteArray'):
++            cassandra.deserializers.DesBytesType = cassandra.deserializers.DesBytesTypeByteArray
++        else:
++            del cassandra.deserializers.DesBytesType
++
 +    cassandra.cqltypes.BytesType.deserialize = staticmethod(lambda byts, protocol_version: bytearray(byts))
 +    cassandra.cqltypes.CassandraType.support_empty_values = True
 +
 +
 +def auto_format_udts():
 +    # when we see a new user defined type, set up the shell formatting for it
 +    udt_apply_params = cassandra.cqltypes.UserType.apply_parameters
 +
 +    def new_apply_params(cls, *args, **kwargs):
 +        udt_class = udt_apply_params(*args, **kwargs)
 +        formatter_for(udt_class.typename)(format_value_utype)
 +        return udt_class
 +
 +    cassandra.cqltypes.UserType.udt_apply_parameters = classmethod(new_apply_params)
 +
 +    make_udt_class = cassandra.cqltypes.UserType.make_udt_class
 +
 +    def new_make_udt_class(cls, *args, **kwargs):
 +        udt_class = make_udt_class(*args, **kwargs)
 +        formatter_for(udt_class.tuple_type.__name__)(format_value_utype)
 +        return udt_class
 +
 +    cassandra.cqltypes.UserType.make_udt_class = classmethod(new_make_udt_class)
 +
 +
 +class FrozenType(cassandra.cqltypes._ParameterizedType):
 +    """
 +    Needed until the bundled python driver adds FrozenType.
 +    """
 +    typename = "frozen"
 +    num_subtypes = 1
 +
 +    @classmethod
 +    def deserialize_safe(cls, byts, protocol_version):
 +        subtype, = cls.subtypes
 +        return subtype.from_binary(byts)
 +
 +    @classmethod
 +    def serialize_safe(cls, val, protocol_version):
 +        subtype, = cls.subtypes
 +        return subtype.to_binary(val, protocol_version)
 +
 +
 +class Shell(cmd.Cmd):
 +    custom_prompt = os.getenv('CQLSH_PROMPT', '')
 +    if custom_prompt is not '':
 +        custom_prompt += "\n"
 +    default_prompt = custom_prompt + "cqlsh> "
 +    continue_prompt = "   ... "
 +    keyspace_prompt = custom_prompt + "cqlsh:%s> "
 +    keyspace_continue_prompt = "%s    ... "
 +    show_line_nums = False
 +    debug = False
 +    stop = False
 +    last_hist = None
 +    shunted_query_out = None
 +    use_paging = True
 +
 +    default_page_size = 100
 +
 +    def __init__(self, hostname, port, color=False,
 +                 username=None, password=None, encoding=None, stdin=None, tty=True,
 +                 completekey=DEFAULT_COMPLETEKEY, browser=None, use_conn=None,
 +                 cqlver=DEFAULT_CQLVER, keyspace=None,
 +                 tracing_enabled=False, expand_enabled=False,
 +                 display_nanotime_format=DEFAULT_NANOTIME_FORMAT,
 +                 display_timestamp_format=DEFAULT_TIMESTAMP_FORMAT,
 +                 display_date_format=DEFAULT_DATE_FORMAT,
 +                 display_float_precision=DEFAULT_FLOAT_PRECISION,
 +                 display_timezone=None,
 +                 max_trace_wait=DEFAULT_MAX_TRACE_WAIT,
 +                 ssl=False,
 +                 single_statement=None,
 +                 request_timeout=DEFAULT_REQUEST_TIMEOUT_SECONDS,
 +                 protocol_version=DEFAULT_PROTOCOL_VERSION,
 +                 connect_timeout=DEFAULT_CONNECT_TIMEOUT_SECONDS):
 +        cmd.Cmd.__init__(self, completekey=completekey)
 +        self.hostname = hostname
 +        self.port = port
 +        self.auth_provider = None
 +        if username:
 +            if not password:
 +                password = getpass.getpass()
 +            self.auth_provider = PlainTextAuthProvider(username=username, password=password)
 +        self.username = username
 +        self.keyspace = keyspace
 +        self.ssl = ssl
 +        self.tracing_enabled = tracing_enabled
 +        self.page_size = self.default_page_size
 +        self.expand_enabled = expand_enabled
 +        if use_conn:
 +            self.conn = use_conn
 +        else:
 +            self.conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=cqlver,
 +                                protocol_version=protocol_version,
 +                                auth_provider=self.auth_provider,
 +                                ssl_options=sslhandling.ssl_settings(hostname, CONFIG_FILE) if ssl else None,
 +                                load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]),
 +                                control_connection_timeout=connect_timeout,
 +                                connect_timeout=connect_timeout)
 +        self.owns_connection = not use_conn
 +        self.set_expanded_cql_version(cqlver)
 +
 +        if keyspace:
 +            self.session = self.conn.connect(keyspace)
 +        else:
 +            self.session = self.conn.connect()
 +
 +        if browser == "":
 +            browser = None
 +        self.browser = browser
 +        self.color = color
 +
 +        self.display_nanotime_format = display_nanotime_format
 +        self.display_timestamp_format = display_timestamp_format
 +        self.display_date_format = display_date_format
 +
 +        self.display_float_precision = display_float_precision
 +
 +        self.display_timezone = display_timezone
 +
 +        # If there is no schema metadata present (due to a schema mismatch), force schema refresh
 +        if not self.conn.metadata.keyspaces:
 +            self.refresh_schema_metadata_best_effort()
 +
 +        self.session.default_timeout = request_timeout
 +        self.session.row_factory = ordered_dict_factory
 +        self.session.default_consistency_level = cassandra.ConsistencyLevel.ONE
 +        self.get_connection_versions()
 +
 +        self.current_keyspace = keyspace
 +
 +        self.display_timestamp_format = display_timestamp_format
 +        self.display_nanotime_format = display_nanotime_format
 +        self.display_date_format = display_date_format
 +
 +        self.max_trace_wait = max_trace_wait
 +        self.session.max_trace_wait = max_trace_wait
 +
 +        self.tty = tty
 +        self.encoding = encoding
 +        self.check_windows_encoding()
 +
 +        self.output_codec = codecs.lookup(encoding)
 +
 +        self.statement = StringIO()
 +        self.lineno = 1
 +        self.in_comment = False
 +
 +        self.prompt = ''
 +        if stdin is None:
 +            stdin = sys.stdin
 +
 +        if tty:
 +            self.reset_prompt()
 +            self.report_connection()
 +            print 'Use HELP for help.'
 +        else:
 +            self.show_line_nums = True
 +        self.stdin = stdin
 +        self.query_out = sys.stdout
 +        self.consistency_level = cassandra.ConsistencyLevel.ONE
 +        self.serial_consistency_level = cassandra.ConsistencyLevel.SERIAL
 +
 +        self.empty_lines = 0
 +        self.statement_error = False
 +        self.single_statement = single_statement
 +
 +    @property
 +    def is_using_utf8(self):
 +        # utf8 encodings from https://docs.python.org/{2,3}/library/codecs.html
 +        return self.encoding.replace('-', '_').lower() in ['utf', 'utf_8', 'u8', 'utf8', CP65001]
 +
 +    def check_windows_encoding(self):
 +        if is_win and os.name == 'nt' and self.tty and \
 +           self.is_using_utf8 and sys.stdout.encoding != CP65001:
 +            self.printerr("\nWARNING: console codepage must be set to cp65001 "
 +                          "to support {} encoding on Windows platforms.\n"
 +                          "If you experience encoding problems, change your console"
 +                          " codepage with 'chcp 65001' before starting cqlsh.\n".format(self.encoding))
 +
 +    def refresh_schema_metadata_best_effort(self):
 +        try:
 +            self.conn.refresh_schema_metadata(5)  # will throw exception if there is a schema mismatch
 +        except Exception:
 +            self.printerr("Warning: schema version mismatch detected, which might be caused by DOWN nodes; if "
 +                          "this is not the case, check the schema versions of your nodes in system.local and "
 +                          "system.peers.")
 +            self.conn.refresh_schema_metadata(0)
 +
 +    def set_expanded_cql_version(self, ver):
 +        ver, vertuple = full_cql_version(ver)
 +        self.cql_version = ver
 +        self.cql_ver_tuple = vertuple
 +
 +    def cqlver_atleast(self, major, minor=0, patch=0):
 +        return self.cql_ver_tuple[:3] >= (major, minor, patch)
 +
 +    def myformat_value(self, val, **kwargs):
 +        if isinstance(val, DecodeError):
 +            self.decoding_errors.append(val)
 +        try:
 +            dtformats = DateTimeFormat(timestamp_format=self.display_timestamp_format,
 +                                       date_format=self.display_date_format, nanotime_format=self.display_nanotime_format,
 +                                       timezone=self.display_timezone)
 +            return format_value(val, self.output_codec.name,
 +                                addcolor=self.color, date_time_format=dtformats,
 +                                float_precision=self.display_float_precision, **kwargs)
 +        except Exception, e:
 +            err = FormatError(val, e)
 +            self.decoding_errors.append(err)
 +            return format_value(err, self.output_codec.name, addcolor=self.color)
 +
 +    def myformat_colname(self, name, table_meta=None):
 +        column_colors = COLUMN_NAME_COLORS.copy()
 +        # check column role and color appropriately
 +        if table_meta:
 +            if name in [col.name for col in table_meta.partition_key]:
 +                column_colors.default_factory = lambda: RED
 +            elif name in [col.name for col in table_meta.clustering_key]:
 +                column_colors.default_factory = lambda: CYAN
 +        return self.myformat_value(name, colormap=column_colors)
 +
 +    def report_connection(self):
 +        self.show_host()
 +        self.show_version()
 +
 +    def show_host(self):
 +        print "Connected to %s at %s:%d." % \
 +            (self.applycolor(self.get_cluster_name(), BLUE),
 +              self.hostname,
 +              self.port)
 +
 +    def show_version(self):
 +        vers = self.connection_versions.copy()
 +        vers['shver'] = version
 +        # system.Versions['cql'] apparently does not reflect changes with
 +        # set_cql_version.
 +        vers['cql'] = self.cql_version
 +        print "[cqlsh %(shver)s | Cassandra %(build)s | CQL spec %(cql)s | Native protocol v%(protocol)s]" % vers
 +
 +    def show_session(self, sessionid, partial_session=False):
 +        print_trace_session(self, self.session, sessionid, partial_session)
 +
 +    def get_connection_versions(self):
 +        result, = self.session.execute("select * from system.local where key = 'local'")
 +        vers = {
 +            'build': result['release_version'],
 +            'protocol': result['native_protocol_version'],
 +            'cql': result['cql_version'],
 +        }
 +        self.connection_versions = vers
 +
 +    def get_keyspace_names(self):
 +        return map(str, self.conn.metadata.keyspaces.keys())
 +
 +    def get_columnfamily_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return map(str, self.get_keyspace_meta(ksname).tables.keys())
 +
 +    def get_index_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return map(str, self.get_keyspace_meta(ksname).indexes.keys())
 +
 +    def get_column_names(self, ksname, cfname):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        layout = self.get_table_meta(ksname, cfname)
 +        return [str(col) for col in layout.columns]
 +
 +    def get_usertype_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return self.get_keyspace_meta(ksname).user_types.keys()
 +
 +    def get_usertype_layout(self, ksname, typename):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        ks_meta = self.get_keyspace_meta(ksname)
 +
 +        try:
 +            user_type = ks_meta.user_types[typename]
 +        except KeyError:
 +            raise UserTypeNotFound("User type %r not found" % typename)
 +
 +        return [(field_name, field_type.cql_parameterized_type())
 +                for field_name, field_type in zip(user_type.field_names, user_type.field_types)]
 +
 +    def get_userfunction_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return map(lambda f: f.name, self.get_keyspace_meta(ksname).functions.values())
 +
 +    def get_useraggregate_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return map(lambda f: f.name, self.get_keyspace_meta(ksname).aggregates.values())
 +
 +    def get_cluster_name(self):
 +        return self.conn.metadata.cluster_name
 +
 +    def get_partitioner(self):
 +        return self.conn.metadata.partitioner
 +
 +    def get_keyspace_meta(self, ksname):
 +        if ksname not in self.conn.metadata.keyspaces:
 +            raise KeyspaceNotFound('Keyspace %r not found.' % ksname)
 +        return self.conn.metadata.keyspaces[ksname]
 +
 +    def get_keyspaces(self):
 +        return self.conn.metadata.keyspaces.values()
 +
 +    def get_ring(self, ks):
 +        self.conn.metadata.token_map.rebuild_keyspace(ks, build_if_absent=True)
 +        return self.conn.metadata.token_map.tokens_to_hosts_by_ks[ks]
 +
 +    def get_table_meta(self, ksname, tablename):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        ksmeta = self.get_keyspace_meta(ksname)
 +
 +        if tablename not in ksmeta.tables:
 +            if ksname == 'system_auth' and tablename in ['roles', 'role_permissions']:
 +                self.get_fake_auth_table_meta(ksname, tablename)
 +            else:
 +                raise ColumnFamilyNotFound("Column family %r not found" % tablename)
 +        else:
 +            return ksmeta.tables[tablename]
 +
 +    def get_fake_auth_table_meta(self, ksname, tablename):
 +        # may be using external auth implementation so internal tables
 +        # aren't actually defined in schema. In this case, we'll fake
 +        # them up
 +        if tablename == 'roles':
 +            ks_meta = KeyspaceMetadata(ksname, True, None, None)
 +            table_meta = TableMetadata(ks_meta, 'roles')
 +            table_meta.columns['role'] = ColumnMetadata(table_meta, 'role', cassandra.cqltypes.UTF8Type)
 +            table_meta.columns['is_superuser'] = ColumnMetadata(table_meta, 'is_superuser', cassandra.cqltypes.BooleanType)
 +            table_meta.columns['can_login'] = ColumnMetadata(table_meta, 'can_login', cassandra.cqltypes.BooleanType)
 +        elif tablename == 'role_permissions':
 +            ks_meta = KeyspaceMetadata(ksname, True, None, None)
 +            table_meta = TableMetadata(ks_meta, 'role_permissions')
 +            table_meta.columns['role'] = ColumnMetadata(table_meta, 'role', cassandra.cqltypes.UTF8Type)
 +            table_meta.columns['resource'] = ColumnMetadata(table_meta, 'resource', cassandra.cqltypes.UTF8Type)
 +            table_meta.columns['permission'] = ColumnMetadata(table_meta, 'permission', cassandra.cqltypes.UTF8Type)
 +        else:
 +            raise ColumnFamilyNotFound("Column family %r not found" % tablename)
 +
 +    def get_index_meta(self, ksname, idxname):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        ksmeta = self.get_keyspace_meta(ksname)
 +
 +        if idxname not in ksmeta.indexes:
 +            raise IndexNotFound("Index %r not found" % idxname)
 +
 +        return ksmeta.indexes[idxname]
 +
 +    def get_object_meta(self, ks, name):
 +        if name is None:
 +            if ks and ks in self.conn.metadata.keyspaces:
 +                return self.conn.metadata.keyspaces[ks]
 +            elif self.current_keyspace is None:
 +                raise ObjectNotFound("%r not found in keyspaces" % (ks))
 +            else:
 +                name = ks
 +                ks = self.current_keyspace
 +
 +        if ks is None:
 +            ks = self.current_keyspace
 +
 +        ksmeta = self.get_keyspace_meta(ks)
 +
 +        if name in ksmeta.tables:
 +            return ksmeta.tables[name]
 +        elif name in ksmeta.indexes:
 +            return ksmeta.indexes[name]
 +
 +        raise ObjectNotFound("%r not found in keyspace %r" % (name, ks))
 +
 +    def get_usertypes_meta(self):
 +        data = self.session.execute("select * from system.schema_usertypes")
 +        if not data:
 +            return cql3handling.UserTypesMeta({})
 +
 +        return cql3handling.UserTypesMeta.from_layout(data)
 +
 +    def get_trigger_names(self, ksname=None):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +
 +        return [trigger.name
 +                for table in self.get_keyspace_meta(ksname).tables.values()
 +                for trigger in table.triggers.values()]
 +
 +    def reset_statement(self):
 +        self.reset_prompt()
 +        self.statement.truncate(0)
 +        self.empty_lines = 0
 +
 +    def reset_prompt(self):
 +        if self.current_keyspace is None:
 +            self.set_prompt(self.default_prompt, True)
 +        else:
 +            self.set_prompt(self.keyspace_prompt % self.current_keyspace, True)
 +
 +    def set_continue_prompt(self):
 +        if self.empty_lines >= 3:
 +            self.set_prompt("Statements are terminated with a ';'.  You can press CTRL-C to cancel an incomplete statement.")
 +            self.empty_lines = 0
 +            return
 +        if self.current_keyspace is None:
 +            self.set_prompt(self.continue_prompt)
 +        else:
 +            spaces = ' ' * len(str(self.current_keyspace))
 +            self.set_prompt(self.keyspace_continue_prompt % spaces)
 +        self.empty_lines = self.empty_lines + 1 if not self.lastcmd else 0
 +
 +    @contextmanager
 +    def prepare_loop(self):
 +        readline = None
 +        if self.tty and self.completekey:
 +            try:
 +                import readline
 +            except ImportError:
 +                if is_win:
 +                    print "WARNING: pyreadline dependency missing.  Install to enable tab completion."
 +                pass
 +            else:
 +                old_completer = readline.get_completer()
 +                readline.set_completer(self.complete)
 +                if readline.__doc__ is not None and 'libedit' in readline.__doc__:
 +                    readline.parse_and_bind("bind -e")
 +                    readline.parse_and_bind("bind '" + self.completekey + "' rl_complete")
 +                    readline.parse_and_bind("bind ^R em-inc-search-prev")
 +                else:
 +                    readline.parse_and_bind(self.completekey + ": complete")
 +        try:
 +            yield
 +        finally:
 +            if readline is not None:
 +                readline.set_completer(old_completer)
 +
 +    def get_input_line(self, prompt=''):
 +        if self.tty:
 +            try:
 +                self.lastcmd = raw_input(prompt).decode(self.encoding)
 +            except UnicodeDecodeError:
 +                self.lastcmd = ''
 +                traceback.print_exc()
 +                self.check_windows_encoding()
 +            line = self.lastcmd + '\n'
 +        else:
 +            self.lastcmd = self.stdin.readline()
 +            line = self.lastcmd
 +            if not len(line):
 +                raise EOFError
 +        self.lineno += 1
 +        return line
 +
 +    def use_stdin_reader(self, until='', prompt=''):
 +        until += '\n'
 +        while True:
 +            try:
 +                newline = self.get_input_line(prompt=prompt)
 +            except EOFError:
 +                return
 +            if newline == until:
 +                return
 +            yield newline
 +
 +    def cmdloop(self):
 +        """
 +        Adapted from cmd.Cmd's version, because there is literally no way with
 +        cmd.Cmd.cmdloop() to tell the difference between "EOF" showing up in
 +        input and an actual EOF.
 +        """
 +        with self.prepare_loop():
 +            while not self.stop:
 +                try:
 +                    if self.single_statement:
 +                        line = self.single_statement
 +                        self.stop = True
 +                    else:
 +                        line = self.get_input_line(self.prompt)
 +                    self.statement.write(line)
 +                    if self.onecmd(self.statement.getvalue()):
 +                        self.reset_statement()
 +                except EOFError:
 +                    self.handle_eof()
 +                except CQL_ERRORS, cqlerr:
 +                    self.printerr(str(cqlerr))
 +                except KeyboardInterrupt:
 +                    self.reset_statement()
 +                    print
 +
 +    def onecmd(self, statementtext):
 +        """
 +        Returns true if the statement is complete and was handled (meaning it
 +        can be reset).
 +        """
 +
 +        try:
 +            statements, endtoken_escaped = cqlruleset.cql_split_statements(statementtext)
 +        except pylexotron.LexingError, e:
 +            if self.show_line_nums:
 +                self.printerr('Invalid syntax at char %d' % (e.charnum,))
 +            else:
 +                self.printerr('Invalid syntax at line %d, char %d'
 +                              % (e.linenum, e.charnum))
 +            statementline = statementtext.split('\n')[e.linenum - 1]
 +            self.printerr('  %s' % statementline)
 +            self.printerr(' %s^' % (' ' * e.charnum))
 +            return True
 +
 +        while statements and not statements[-1]:
 +            statements = statements[:-1]
 +        if not statements:
 +            return True
 +        if endtoken_escaped or statements[-1][-1][0] != 'endtoken':
 +            self.set_continue_prompt()
 +            return
 +        for st in statements:
 +            try:
 +                self.handle_statement(st, statementtext)
 +            except Exception, e:
 +                if self.debug:
 +                    traceback.print_exc()
 +                else:
 +                    self.printerr(e)
 +        return True
 +
 +    def handle_eof(self):
 +        if self.tty:
 +            print
 +        statement = self.statement.getvalue()
 +        if statement.strip():
 +            if not self.onecmd(statement):
 +                self.printerr('Incomplete statement at end of file')
 +        self.do_exit()
 +
 +    def handle_statement(self, tokens, srcstr):
 +        # Concat multi-line statements and insert into history
 +        if readline is not None:
 +            nl_count = srcstr.count("\n")
 +
 +            new_hist = srcstr.replace("\n", " ").rstrip()
 +
 +            if nl_count > 1 and self.last_hist != new_hist:
 +                readline.add_history(new_hist.encode(self.encoding))
 +
 +            self.last_hist = new_hist
 +        cmdword = tokens[0][1]
 +        if cmdword == '?':
 +            cmdword = 'help'
 +        custom_handler = getattr(self, 'do_' + cmdword.lower(), None)
 +        if custom_handler:
 +            parsed = cqlruleset.cql_whole_parse_tokens(tokens, srcstr=srcstr,
 +                                                       startsymbol='cqlshCommand')
 +            if parsed and not parsed.remainder:
 +                # successful complete parse
 +                return custom_handler(parsed)
 +            else:
 +                return self.handle_parse_error(cmdword, tokens, parsed, srcstr)
 +        return self.perform_statement(cqlruleset.cql_extract_orig(tokens, srcstr))
 +
 +    def handle_parse_error(self, cmdword, tokens, parsed, srcstr):
 +        if cmdword.lower() in ('select', 'insert', 'update', 'delete', 'truncate',
 +                               'create', 'drop', 'alter', 'grant', 'revoke',
 +                               'batch', 'list'):
 +            # hey, maybe they know about some new syntax we don't. type
 +            # assumptions won't work, but maybe the query will.
 +            return self.perform_statement(cqlruleset.cql_extract_orig(tokens, srcstr))
 +        if parsed:
 +            self.printerr('Improper %s command (problem at %r).' % (cmdword, parsed.remainder[0]))
 +        else:
 +            self.printerr('Improper %s command.' % cmdword)
 +
 +    def do_use(self, parsed):
 +        ksname = parsed.get_binding('ksname')
 +        success, _ = self.perform_simple_statement(SimpleStatement(parsed.extract_orig()))
 +        if success:
 +            if ksname[0] == '"' and ksname[-1] == '"':
 +                self.current_keyspace = self.cql_unprotect_name(ksname)
 +            else:
 +                self.current_keyspace = ksname.lower()
 +
 +    def do_select(self, parsed):
 +        tracing_was_enabled = self.tracing_enabled
 +        ksname = parsed.get_binding('ksname')
 +        stop_tracing = ksname == 'system_traces' or (ksname is None and self.current_keyspace == 'system_traces')
 +        self.tracing_enabled = self.tracing_enabled and not stop_tracing
 +        statement = parsed.extract_orig()
 +        self.perform_statement(statement)
 +        self.tracing_enabled = tracing_was_enabled
 +
 +    def perform_statement(self, statement):
 +        stmt = SimpleStatement(statement, consistency_level=self.consistency_level, serial_consistency_level=self.serial_consistency_level, fetch_size=self.page_size if self.use_paging else None)
 +        success, future = self.perform_simple_statement(stmt)
 +
 +        if future:
 +            if future.warnings:
 +                self.print_warnings(future.warnings)
 +
 +            if self.tracing_enabled:
 +                try:
 +                    for trace in future.get_all_query_traces(self.max_trace_wait):
 +                        print_trace(self, trace)
 +                except TraceUnavailable:
 +                    msg = "Statement trace did not complete within %d seconds; trace data may be incomplete." % (self.session.max_trace_wait,)
 +                    self.writeresult(msg, color=RED)
 +                    for trace_id in future.get_query_trace_ids():
 +                        self.show_session(trace_id, partial_session=True)
 +                except Exception, err:
 +                    self.printerr("Unable to fetch query trace: %s" % (str(err),))
 +
 +        return success
 +
 +    def parse_for_table_meta(self, query_string):
 +        try:
 +            parsed = cqlruleset.cql_parse(query_string)[1]
 +        except IndexError:
 +            return None
 +        ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +        cf = self.cql_unprotect_name(parsed.get_binding('cfname'))
 +        return self.get_table_meta(ks, cf)
 +
 +    def perform_simple_statement(self, statement):
 +        if not statement:
 +            return False, None
 +
 +        while True:
 +            try:
 +                future = self.session.execute_async(statement, trace=self.tracing_enabled)
 +                result = future.result()
 +                break
 +            except cassandra.OperationTimedOut, err:
 +                self.refresh_schema_metadata_best_effort()
 +                self.printerr(str(err.__class__.__name__) + ": " + str(err))
 +                return False, None
 +            except CQL_ERRORS, err:
 +                self.printerr(str(err.__class__.__name__) + ": " + str(err))
 +                return False, None
 +            except Exception, err:
 +                import traceback
 +                self.printerr(traceback.format_exc())
 +                return False, None
 +
 +        if statement.query_string[:6].lower() == 'select':
 +            self.print_result(result, self.parse_for_table_meta(statement.query_string))
 +        elif statement.query_string.lower().startswith("list users") or statement.query_string.lower().startswith("list roles"):
 +            self.print_result(result, self.get_table_meta('system_auth', 'roles'))
 +        elif statement.query_string.lower().startswith("list"):
 +            self.print_result(result, self.get_table_meta('system_auth', 'role_permissions'))
 +        elif result:
 +            # CAS INSERT/UPDATE
 +            self.writeresult("")
 +            self.print_static_result(result.column_names, list(result), self.parse_for_table_meta(statement.query_string))
 +        self.flush_output()
 +        return True, future
 +
 +    def print_result(self, result, table_meta):
 +        self.decoding_errors = []
 +
 +        self.writeresult("")
 +        if result.has_more_pages and self.tty:
 +            num_rows = 0
 +            while True:
 +                page = result.current_rows
 +                if page:
 +                    num_rows += len(page)
 +                    self.print_static_result(result.column_names, page, table_meta)
 +                if result.has_more_pages:
 +                    raw_input("---MORE---")
 +                    result.fetch_next_page()
 +                else:
 +                    break
 +        else:
 +            rows = list(result)
 +            num_rows = len(rows)
 +            self.print_static_result(result.column_names, rows, table_meta)
 +        self.writeresult("(%d rows)" % num_rows)
 +
 +        if self.decoding_errors:
 +            for err in self.decoding_errors[:2]:
 +                self.writeresult(err.message(), color=RED)
 +            if len(self.decoding_errors) > 2:
 +                self.writeresult('%d more decoding errors suppressed.'
 +                                 % (len(self.decoding_errors) - 2), color=RED)
 +
 +    def print_static_result(self, column_names, rows, table_meta):
 +        if not column_names and not table_meta:
 +            return
 +        column_names = column_names or table_meta.columns.keys()
 +        formatted_names = [self.myformat_colname(name, table_meta) for name in column_names]
 +        formatted_values = [map(self.myformat_value, row.values()) for row in rows]
 +
 +        if self.expand_enabled:
 +            self.print_formatted_result_vertically(formatted_names, formatted_values)
 +        else:
 +            self.print_formatted_result(formatted_names, formatted_values)
 +
 +    def print_formatted_result(self, formatted_names, formatted_values):
 +        # determine column widths
 +        widths = [n.displaywidth for n in formatted_names]
 +        if formatted_values is not None:
 +            for fmtrow in formatted_values:
 +                for num, col in enumerate(fmtrow):
 +                    widths[num] = max(widths[num], col.displaywidth)
 +
 +        # print header
 +        header = ' | '.join(hdr.ljust(w, color=self.color) for (hdr, w) in zip(formatted_names, widths))
 +        self.writeresult(' ' + header.rstrip())
 +        self.writeresult('-%s-' % '-+-'.join('-' * w for w in widths))
 +
 +        # stop if there are no rows
 +        if formatted_values is None:
 +            self.writeresult("")
 +            return
 +
 +        # print row data
 +        for row in formatted_values:
 +            line = ' | '.join(col.rjust(w, color=self.color) for (col, w) in zip(row, widths))
 +            self.writeresult(' ' + line)
 +
 +        self.writeresult("")
 +
 +    def print_formatted_result_vertically(self, formatted_names, formatted_values):
 +        max_col_width = max([n.displaywidth for n in formatted_names])
 +        max_val_width = max([n.displaywidth for row in formatted_values for n in row])
 +
 +        # for each row returned, list all the column-value pairs
 +        for row_id, row in enumerate(formatted_values):
 +            self.writeresult("@ Row %d" % (row_id + 1))
 +            self.writeresult('-%s-' % '-+-'.join(['-' * max_col_width, '-' * max_val_width]))
 +            for field_id, field in enumerate(row):
 +                column = formatted_names[field_id].ljust(max_col_width, color=self.color)
 +                value = field.ljust(field.displaywidth, color=self.color)
 +                self.writeresult(' ' + " | ".join([column, value]))
 +            self.writeresult('')
 +
 +    def print_warnings(self, warnings):
 +        if warnings is None or len(warnings) == 0:
 +            return
 +
 +        self.writeresult('')
 +        self.writeresult('Warnings :')
 +        for warning in warnings:
 +            self.writeresult(warning)
 +            self.writeresult('')
 +
 +    def emptyline(self):
 +        pass
 +
 +    def parseline(self, line):
 +        # this shouldn't be needed
 +        raise NotImplementedError
 +
 +    def complete(self, text, state):
 +        if readline is None:
 +            return
 +        if state == 0:
 +            try:
 +                self.completion_matches = self.find_completions(text)
 +            except Exception:
 +                if debug_completion:
 +                    import traceback
 +                    traceback.print_exc()
 +                else:
 +                    raise
 +        try:
 +            return self.completion_matches[state]
 +        except IndexError:
 +            return None
 +
 +    def find_completions(self, text):
 +        curline = readline.get_line_buffer()
 +        prevlines = self.statement.getvalue()
 +        wholestmt = prevlines + curline
 +        begidx = readline.get_begidx() + len(prevlines)
 +        stuff_to_complete = wholestmt[:begidx]
 +        return cqlruleset.cql_complete(stuff_to_complete, text, cassandra_conn=self,
 +                                       debug=debug_completion, startsymbol='cqlshCommand')
 +
 +    def set_prompt(self, prompt, prepend_user=False):
 +        if prepend_user and self.username:
 +            self.prompt = "%s@%s" % (self.username, prompt)
 +            return
 +        self.prompt = prompt
 +
 +    def cql_unprotect_name(self, namestr):
 +        if namestr is None:
 +            return
 +        return cqlruleset.dequote_name(namestr)
 +
 +    def cql_unprotect_value(self, valstr):
 +        if valstr is not None:
 +            return cqlruleset.dequote_value(valstr)
 +
 +    def print_recreate_keyspace(self, ksdef, out):
 +        out.write(ksdef.export_as_string())
 +        out.write("\n")
 +
 +    def print_recreate_columnfamily(self, ksname, cfname, out):
 +        """
 +        Output CQL commands which should be pasteable back into a CQL session
 +        to recreate the given table.
 +
 +        Writes output to the given out stream.
 +        """
 +        out.write(self.get_table_meta(ksname, cfname).export_as_string())
 +        out.write("\n")
 +
 +    def print_recreate_index(self, ksname, idxname, out):
 +        """
 +        Output CQL commands which should be pasteable back into a CQL session
 +        to recreate the given index.
 +
 +        Writes output to the given out stream.
 +        """
 +        out.write(self.get_index_meta(ksname, idxname).export_as_string())
 +        out.write("\n")
 +
 +    def print_recreate_object(self, ks, name, out):
 +        """
 +        Output CQL commands which should be pasteable back into a CQL session
 +        to recreate the given object (ks, table or index).
 +
 +        Writes output to the given out stream.
 +        """
 +        out.write(self.get_object_meta(ks, name).export_as_string())
 +        out.write("\n")
 +
 +    def describe_keyspaces(self):
 +        print
 +        cmd.Cmd.columnize(self, protect_names(self.get_keyspace_names()))
 +        print
 +
 +    def describe_keyspace(self, ksname):
 +        print
 +        self.print_recreate_keyspace(self.get_keyspace_meta(ksname), sys.stdout)
 +        print
 +
 +    def describe_columnfamily(self, ksname, cfname):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        if ksname is None:
 +            raise NoKeyspaceError("No keyspace specified and no current keyspace")
 +        print
 +        self.print_recreate_columnfamily(ksname, cfname, sys.stdout)
 +        print
 +
 +    def describe_index(self, ksname, idxname):
 +        print
 +        self.print_recreate_index(ksname, idxname, sys.stdout)
 +        print
 +
 +    def describe_object(self, ks, name):
 +        print
 +        self.print_recreate_object(ks, name, sys.stdout)
 +        print
 +
 +    def describe_columnfamilies(self, ksname):
 +        print
 +        if ksname is None:
 +            for k in self.get_keyspaces():
 +                name = protect_name(k.name)
 +                print 'Keyspace %s' % (name,)
 +                print '---------%s' % ('-' * len(name))
 +                cmd.Cmd.columnize(self, protect_names(self.get_columnfamily_names(k.name)))
 +                print
 +        else:
 +            cmd.Cmd.columnize(self, protect_names(self.get_columnfamily_names(ksname)))
 +            print
 +
 +    def describe_functions(self, ksname):
 +        print
 +        if ksname is None:
 +            for ksmeta in self.get_keyspaces():
 +                name = protect_name(ksmeta.name)
 +                print 'Keyspace %s' % (name,)
 +                print '---------%s' % ('-' * len(name))
 +                cmd.Cmd.columnize(self, protect_names(ksmeta.functions.keys()))
 +                print
 +        else:
 +            ksmeta = self.get_keyspace_meta(ksname)
 +            cmd.Cmd.columnize(self, protect_names(ksmeta.functions.keys()))
 +            print
 +
 +    def describe_function(self, ksname, functionname):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        if ksname is None:
 +            raise NoKeyspaceError("No keyspace specified and no current keyspace")
 +        print
 +        ksmeta = self.get_keyspace_meta(ksname)
 +        functions = filter(lambda f: f.name == functionname, ksmeta.functions.values())
 +        if len(functions) == 0:
 +            raise FunctionNotFound("User defined function %r not found" % functionname)
 +        print "\n\n".join(func.as_cql_query(formatted=True) for func in functions)
 +        print
 +
 +    def describe_aggregates(self, ksname):
 +        print
 +        if ksname is None:
 +            for ksmeta in self.get_keyspaces():
 +                name = protect_name(ksmeta.name)
 +                print 'Keyspace %s' % (name,)
 +                print '---------%s' % ('-' * len(name))
 +                cmd.Cmd.columnize(self, protect_names(ksmeta.aggregates.keys()))
 +                print
 +        else:
 +            ksmeta = self.get_keyspace_meta(ksname)
 +            cmd.Cmd.columnize(self, protect_names(ksmeta.aggregates.keys()))
 +            print
 +
 +    def describe_aggregate(self, ksname, aggregatename):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        if ksname is None:
 +            raise NoKeyspaceError("No keyspace specified and no current keyspace")
 +        print
 +        ksmeta = self.get_keyspace_meta(ksname)
 +        aggregates = filter(lambda f: f.name == aggregatename, ksmeta.aggregates.values())
 +        if len(aggregates) == 0:
 +            raise FunctionNotFound("User defined aggregate %r not found" % aggregatename)
 +        print "\n\n".join(aggr.as_cql_query(formatted=True) for aggr in aggregates)
 +        print
 +
 +    def describe_usertypes(self, ksname):
 +        print
 +        if ksname is None:
 +            for ksmeta in self.get_keyspaces():
 +                name = protect_name(ksmeta.name)
 +                print 'Keyspace %s' % (name,)
 +                print '---------%s' % ('-' * len(name))
 +                cmd.Cmd.columnize(self, protect_names(ksmeta.user_types.keys()))
 +                print
 +        else:
 +            ksmeta = self.get_keyspace_meta(ksname)
 +            cmd.Cmd.columnize(self, protect_names(ksmeta.user_types.keys()))
 +            print
 +
 +    def describe_usertype(self, ksname, typename):
 +        if ksname is None:
 +            ksname = self.current_keyspace
 +        if ksname is None:
 +            raise NoKeyspaceError("No keyspace specified and no current keyspace")
 +        print
 +        ksmeta = self.get_keyspace_meta(ksname)
 +        try:
 +            usertype = ksmeta.user_types[typename]
 +        except KeyError:
 +            raise UserTypeNotFound("User type %r not found" % typename)
 +        print usertype.as_cql_query(formatted=True)
 +        print
 +
 +    def describe_cluster(self):
 +        print '\nCluster: %s' % self.get_cluster_name()
 +        p = trim_if_present(self.get_partitioner(), 'org.apache.cassandra.dht.')
 +        print 'Partitioner: %s\n' % p
 +        # TODO: snitch?
 +        # snitch = trim_if_present(self.get_snitch(), 'org.apache.cassandra.locator.')
 +        # print 'Snitch: %s\n' % snitch
 +        if self.current_keyspace is not None and self.current_keyspace != 'system':
 +            print "Range ownership:"
 +            ring = self.get_ring(self.current_keyspace)
 +            for entry in ring.items():
 +                print ' %39s  [%s]' % (str(entry[0].value), ', '.join([host.address for host in entry[1]]))
 +            print
 +
 +    def describe_schema(self, include_system=False):
 +        print
 +        for k in self.get_keyspaces():
 +            if include_system or k.name not in cql3handling.SYSTEM_KEYSPACES:
 +                self.print_recreate_keyspace(k, sys.stdout)
 +                print
 +
 +    def do_describe(self, parsed):
 +        """
 +        DESCRIBE [cqlsh only]
 +
 +        (DESC may be used as a shorthand.)
 +
 +          Outputs information about the connected Cassandra cluster, or about
 +          the data objects stored in the cluster. Use in one of the following ways:
 +
 +        DESCRIBE KEYSPACES
 +
 +          Output the names of all keyspaces.
 +
 +        DESCRIBE KEYSPACE [<keyspacename>]
 +
 +          Output CQL commands that could be used to recreate the given keyspace,
 +          and the objects in it (such as tables, types, functions, etc.).
 +          In some cases, as the CQL interface matures, there will be some metadata
 +          about a keyspace that is not representable with CQL. That metadata will not be shown.
 +
 +          The '<keyspacename>' argument may be omitted, in which case the current
 +          keyspace will be described.
 +
 +        DESCRIBE TABLES
 +
 +          Output the names of all tables in the current keyspace, or in all
 +          keyspaces if there is no current keyspace.
 +
 +        DESCRIBE TABLE [<keyspace>.]<tablename>
 +
 +          Output CQL commands that could be used to recreate the given table.
 +          In some cases, as above, there may be table metadata which is not
 +          representable and which will not be shown.
 +
 +        DESCRIBE INDEX <indexname>
 +
 +          Output the CQL command that could be used to recreate the given index.
 +          In some cases, there may be index metadata which is not representable
 +          and which will not be shown.
 +
 +        DESCRIBE CLUSTER
 +
 +          Output information about the connected Cassandra cluster, such as the
 +          cluster name, and the partitioner and snitch in use. When you are
 +          connected to a non-system keyspace, also shows endpoint-range
 +          ownership information for the Cassandra ring.
 +
 +        DESCRIBE [FULL] SCHEMA
 +
 +          Output CQL commands that could be used to recreate the entire (non-system) schema.
 +          Works as though "DESCRIBE KEYSPACE k" was invoked for each non-system keyspace
 +          k. Use DESCRIBE FULL SCHEMA to include the system keyspaces.
 +
 +        DESCRIBE TYPES
 +
 +          Output the names of all user-defined-types in the current keyspace, or in all
 +          keyspaces if there is no current keyspace.
 +
 +        DESCRIBE TYPE [<keyspace>.]<type>
 +
 +          Output the CQL command that could be used to recreate the given user-defined-type.
 +
 +        DESCRIBE FUNCTIONS
 +
 +          Output the names of all user-defined-functions in the current keyspace, or in all
 +          keyspaces if there is no current keyspace.
 +
 +        DESCRIBE FUNCTION [<keyspace>.]<function>
 +
 +          Output the CQL command that could be used to recreate the given user-defined-function.
 +
 +        DESCRIBE AGGREGATES
 +
 +          Output the names of all user-defined-aggregates in the current keyspace, or in all
 +          keyspaces if there is no current keyspace.
 +
 +        DESCRIBE AGGREGATE [<keyspace>.]<aggregate>
 +
 +          Output the CQL command that could be used to recreate the given user-defined-aggregate.
 +
 +        DESCRIBE <objname>
 +
 +          Output CQL commands that could be used to recreate the entire object schema,
 +          where object can be either a keyspace or a table or an index (in this order).
 +  """
 +        what = parsed.matched[1][1].lower()
 +        if what == 'functions':
 +            self.describe_functions(self.current_keyspace)
 +        elif what == 'function':
 +            ksname = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            functionname = self.cql_unprotect_name(parsed.get_binding('udfname'))
 +            self.describe_function(ksname, functionname)
 +        elif what == 'aggregates':
 +            self.describe_aggregates(self.current_keyspace)
 +        elif what == 'aggregate':
 +            ksname = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            aggregatename = self.cql_unprotect_name(parsed.get_binding('udaname'))
 +            self.describe_aggregate(ksname, aggregatename)
 +        elif what == 'keyspaces':
 +            self.describe_keyspaces()
 +        elif what == 'keyspace':
 +            ksname = self.cql_unprotect_name(parsed.get_binding('ksname', ''))
 +            if not ksname:
 +                ksname = self.current_keyspace
 +                if ksname is None:
 +                    self.printerr('Not in any keyspace.')
 +                    return
 +            self.describe_keyspace(ksname)
 +        elif what in ('columnfamily', 'table'):
 +            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            cf = self.cql_unprotect_name(parsed.get_binding('cfname'))
 +            self.describe_columnfamily(ks, cf)
 +        elif what == 'index':
 +            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            idx = self.cql_unprotect_name(parsed.get_binding('idxname', None))
 +            self.describe_index(ks, idx)
 +        elif what in ('columnfamilies', 'tables'):
 +            self.describe_columnfamilies(self.current_keyspace)
 +        elif what == 'types':
 +            self.describe_usertypes(self.current_keyspace)
 +        elif what == 'type':
 +            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            ut = self.cql_unprotect_name(parsed.get_binding('utname'))
 +            self.describe_usertype(ks, ut)
 +        elif what == 'cluster':
 +            self.describe_cluster()
 +        elif what == 'schema':
 +            self.describe_schema(False)
 +        elif what == 'full' and parsed.matched[2][1].lower() == 'schema':
 +            self.describe_schema(True)
 +        elif what:
 +            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +            name = self.cql_unprotect_name(parsed.get_binding('cfname'))
 +            if not name:
 +                name = self.cql_unprotect_name(parsed.get_binding('idxname', None))
 +            self.describe_object(ks, name)
 +    do_desc = do_describe
 +
 +    def do_copy(self, parsed):
 +        r"""
 +        COPY [cqlsh only]
 +
 +          COPY x FROM: Imports CSV data into a Cassandra table
 +          COPY x TO: Exports data from a Cassandra table in CSV format.
 +
 +        COPY <table_name> [ ( column [, ...] ) ]
 +             FROM ( '<file_pattern_1, file_pattern_2, ... file_pattern_n>' | STDIN )
 +             [ WITH <option>='value' [AND ...] ];
 +
 +        File patterns are either file names or valid python glob expressions, e.g. *.csv or folder/*.csv.
 +
 +        COPY <table_name> [ ( column [, ...] ) ]
 +             TO ( '<filename>' | STDOUT )
 +             [ WITH <option>='value' [AND ...] ];
 +
 +        Available common COPY options and defaults:
 +
 +          DELIMITER=','           - character that appears between records
 +          QUOTE='"'               - quoting character to be used to quote fields
 +          ESCAPE='\'              - character to appear before the QUOTE char when quoted
 +          HEADER=false            - whether to ignore the first line
 +          NULL=''                 - string that represents a null value
 +          DATETIMEFORMAT=         - timestamp strftime format
 +            '%Y-%m-%d %H:%M:%S%z'   defaults to time_format value in cqlshrc
 +          MAXATTEMPTS=5           - the maximum number of attempts per batch or range
 +          REPORTFREQUENCY=0.25    - the frequency with which we display status updates in seconds
 +          DECIMALSEP='.'          - the separator for decimal values
 +          THOUSANDSSEP=''         - the separator for thousands digit groups
 +          BOOLSTYLE='True,False'  - the representation for booleans, case insensitive, specify true followed by false,
 +                                    for example yes,no or 1,0
 +          NUMPROCESSES=n          - the number of worker processes, by default the number of cores minus one
 +                                    capped at 16
 +          CONFIGFILE=''           - a configuration file with the same format as .cqlshrc (see the Python ConfigParser
 +                                    documentation) where you can specify WITH options under the following optional
 +                                    sections: [copy], [copy-to], [copy-from], [copy:ks.table], [copy-to:ks.table],
 +                                    [copy-from:ks.table], where <ks> is your keyspace name and <table> is your table
 +                                    name. Options are read from these sections, in the order specified
 +                                    above, and command line options always override options in configuration files.
 +                                    Depending on the COPY direction, only the relevant copy-from or copy-to sections
 +                                    are used. If no configfile is specified then .cqlshrc is searched instead.
 +          RATEFILE=''             - an optional file where to print the output statistics
 +
 +        Available COPY FROM options and defaults:
 +
-           CHUNKSIZE=1000          - the size of chunks passed to worker processes
++          CHUNKSIZE=5000          - the size of chunks passed to worker processes
 +          INGESTRATE=100000       - an approximate ingest rate in rows per second
-           MINBATCHSIZE=2          - the minimum size of an import batch
++          MINBATCHSIZE=10         - the minimum size of an import batch
 +          MAXBATCHSIZE=20         - the maximum size of an import batch
 +          MAXROWS=-1              - the maximum number of rows, -1 means no maximum
 +          SKIPROWS=0              - the number of rows to skip
 +          SKIPCOLS=''             - a comma separated list of column names to skip
 +          MAXPARSEERRORS=-1       - the maximum global number of parsing errors, -1 means no maximum
 +          MAXINSERTERRORS=-1      - the maximum global number of insert errors, -1 means no maximum
 +          ERRFILE=''              - a file where to store all rows that could not be imported, by default this is
 +                                    import_ks_table.err where <ks> is your keyspace and <table> is your table name.
++          PREPAREDSTATEMENTS=True - whether to use prepared statements when importing, by default True. Set this to
++                                    False if you don't mind shifting data parsing to the cluster. The cluster will also
++                                    have to compile every batch statement. For large and oversized clusters
++                                    this will result in a faster import but for smaller clusters it may generate
++                                    timeouts.
 +
 +        Available COPY TO options and defaults:
 +
 +          ENCODING='utf8'          - encoding for CSV output
 +          PAGESIZE='1000'          - the page size for fetching results
 +          PAGETIMEOUT=10           - the page timeout in seconds for fetching results
 +          BEGINTOKEN=''            - the minimum token string to consider when exporting data
 +          ENDTOKEN=''              - the maximum token string to consider when exporting data
 +          MAXREQUESTS=6            - the maximum number of requests each worker process can work on in parallel
 +          MAXOUTPUTSIZE='-1'       - the maximum size of the output file measured in number of lines,
 +                                     beyond this maximum the output file will be split into segments,
 +                                     -1 means unlimited.
 +
 +        When entering CSV data on STDIN, you can use the sequence "\."
 +        on a line by itself to end the data input.
 +        """
 +
 +        ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
 +        if ks is None:
 +            ks = self.current_keyspace
 +            if ks is None:
 +                raise NoKeyspaceError("Not in any keyspace.")
 +        table = self.cql_unprotect_name(parsed.get_binding('cfname'))
 +        columns = parsed.get_binding('colnames', None)
 +        if columns is not None:
 +            columns = map(self.cql_unprotect_name, columns)
 +        else:
 +            # default to all known columns
 +            columns = self.get_column_names(ks, table)
 +
 +        fname = parsed.get_binding('fname', None)
 +        if fname is not None:
 +            fname = self.cql_unprotect_value(fname)
 +
 +        copyoptnames = map(str.lower, parsed.get_binding('optnames', ()))
 +        copyoptvals = map(self.cql_unprotect_value, parsed.get_binding('optvals', ()))
 +        opts = dict(zip(copyoptnames, copyoptvals))
 +
 +        direction = parsed.get_binding('dir').upper()
 +        if direction == 'FROM':
 +            task = ImportTask(self, ks, table, columns, fname, opts, DEFAULT_PROTOCOL_VERSION, CONFIG_FILE)
 +        elif direction == 'TO':
 +            task = ExportTask(self, ks, table, columns, fname, opts, DEFAULT_PROTOCOL_VERSION, CONFIG_FILE)
 +        else:
 +            raise SyntaxError("Unknown direction %s" % direction)
 +
 +        task.run()
 +
 +    def do_show(self, parsed):
 +        """
 +        SHOW [cqlsh only]
 +
 +          Displays information about the current cqlsh session. Can be called in
 +          the following ways:
 +
 +        SHOW VERSION
 +
 +          Shows the version and build of the connected Cassandra instance, as
 +          well as the versions of the CQL spec and the Thrift protocol that
 +          the connected Cassandra instance understands.
 +
 +        SHOW HOST
 +
 +          Shows where cqlsh is currently connected.
 +
 +        SHOW SESSION <sessionid>
 +
 +          Pretty-prints the requested tracing session.
 +        """
 +        showwhat = parsed.get_binding('what').lower()
 +        if showwhat == 'version':
 +            self.get_connection_versions()
 +            self.show_version()
 +        elif showwhat == 'host':
 +            self.show_host()
 +        elif showwhat.startswith('session'):
 +            session_id = parsed.get_binding('sessionid').lower()
 +            self.show_session(UUID(session_id))
 +        else:
 +            self.printerr('Wait, how do I show %r?' % (showwhat,))
 +
 +    def do_source(self, parsed):
 +        """
 +        SOURCE [cqlsh only]
 +
 +        Executes a file containing CQL statements. Gives the output for each
 +        statement in turn, if any, or any errors that occur along the way.
 +
 +        Errors do NOT abort execution of the CQL source file.
 +
 +        Usage:
 +
 +          SOURCE '<file>';
 +
 +        That is, the path to the file to be executed must be given inside a
 +        string literal. The path is interpreted relative to the current working
 +        directory. The tilde shorthand notation ('~/mydir') is supported for
 +        referring to $HOME.
 +
 +        See also the --file option to cqlsh.
 +        """
 +        fname = parsed.get_binding('fname')
 +        fname = os.path.expanduser(self.cql_unprotect_value(fname))
 +        try:
 +            encoding, bom_size = get_file_encoding_bomsize(fname)
 +            f = codecs.open(fname, 'r', encoding)
 +            f.seek(bom_size)
 +        except IOError, e:
 +            self.printerr('Could not open %r: %s' % (fname, e))
 +            return
 +        subshell = Shell(self.hostname, self.port,
 +                         color=self.color, encoding=self.encoding, stdin=f,
 +                         tty=False, use_conn=self.conn, cqlver=self.cql_version,
 +                         display_timestamp_format=self.display_timestamp_format,
 +                         display_date_format=self.display_date_format,
 +                         display_nanotime_format=self.display_nanotime_format,
 +                         display_float_precision=self.display_float_precision,
 +                         max_trace_wait=self.max_trace_wait)
 +        subshell.cmdloop()
 +        f.close()
 +
 +    def do_capture(self, parsed):
 +        """
 +        CAPTURE [cqlsh only]
 +
 +        Begins capturing command output and appending it to a specified file.
 +        Output will not be shown at the console while it is captured.
 +
 +        Usage:
 +
 +          CAPTURE '<file>';
 +          CAPTURE OFF;
 +          CAPTURE;
 +
 +        That is, the path to the file to be appended to must be given inside a
 +        string literal. The path is interpreted relative to the current working
 +        directory. The tilde shorthand notation ('~/mydir') is supported for
 +        referring to $HOME.
 +
 +        Only query result output is captured. Errors and output from cqlsh-only
 +        commands will still be shown in the cqlsh session.
 +
 +        To stop capturing output and show it in the cqlsh session again, use
 +        CAPTURE OFF.
 +
 +        To inspect the current capture configuration, use CAPTURE with no
 +        arguments.
 +        """
 +        fname = parsed.get_binding('fname')
 +        if fname is None:
 +            if self.shunted_query_out is not None:
 +                print "Currently capturing query output to %r." % (self.query_out.name,)
 +            else:
 +                print "Currently not capturing query output."
 +            return
 +
 +        if fname.upper() == 'OFF':
 +            if self.shunted_query_out is None:
 +                self.printerr('Not currently capturing output.')
 +                return
 +            self.query_out.close()
 +            self.query_out = self.shunted_query_out
 +            self.color = self.shunted_color
 +            self.shunted_query_out = None
 +            del self.shunted_color
 +            return
 +
 +        if self.shunted_query_out is not None:
 +            self.printerr('Already capturing output to %s. Use CAPTURE OFF'
 +                          ' to disable.' % (self.query_out.name,))
 +            return
 +
 +        fname = os.path.expanduser(self.cql_unprotect_value(fname))
 +        try:
 +            f = open(fname, 'a')
 +        except IOError, e:
 +            self.printerr('Could not open %r for append: %s' % (fname, e))
 +            return
 +        self.shunted_query_out = self.query_out
 +        self.shunted_color = self.color
 +        self.query_out = f
 +        self.color = False
 +        print 'Now capturing query output to %r.' % (fname,)
 +
 +    def do_tracing(self, parsed):
 +        """
 +        TRACING [cqlsh]
 +
 +          Enables or disables request tracing.
 +
 +        TRACING ON
 +
 +          Enables tracing for all further requests.
 +
 +        TRACING OFF
 +
 +          Disables tracing.
 +
 +        TRACING
 +
 +          TRACING with no arguments shows the current tracing status.
 +        """
 +        self.tracing_enabled = SwitchCommand("TRACING", "Tracing").execute(self.tracing_enabled, parsed, self.printerr)
 +
 +    def do_expand(self, parsed):
 +        """
 +        EXPAND [cqlsh]
 +
 +          Enables or disables expanded (vertical) output.
 +
 +        EXPAND ON
 +
 +          Enables expanded (vertical) output.
 +
 +        EXPAND OFF
 +
 +          Disables expanded (vertical) output.
 +
 +        EXPAND
 +
 +          EXPAND with no arguments shows the current value of expand setting.
 +        """
 +        self.expand_enabled = SwitchCommand("EXPAND", "Expanded output").execute(self.expand_enabled, parsed, self.printerr)
 +
 +    def do_consistency(self, parsed):
 +        """
 +        CONSISTENCY [cqlsh only]
 +
 +           Overrides default consistency level (default level is ONE).
 +
 +        CONSISTENCY <level>
 +
 +           Sets consistency level for future requests.
 +
 +           Valid consistency levels:
 +
 +           ANY, ONE, TWO, THREE, QUORUM, ALL, LOCAL_ONE, LOCAL_QUORUM, EACH_QUORUM, SERIAL and LOCAL_SERIAL.
 +
 +           SERIAL and LOCAL_SERIAL may be used only for SELECTs; will be rejected with updates.
 +
 +        CONSISTENCY
 +
 +           CONSISTENCY with no arguments shows the current consistency level.
 +        """
 +        level = parsed.get_binding('level')
 +        if level is None:
 +            print 'Current consistency level is %s.' % (cassandra.ConsistencyLevel.value_to_name[self.consistency_level])
 +            return
 +
 +        self.consistency_level = cassandra.ConsistencyLevel.name_to_value[level.upper()]
 +        print 'Consistency level set to %s.' % (level.upper(),)
 +
 +    def do_serial(self, parsed):
 +        """
 +        SERIAL CONSISTENCY [cqlsh only]
 +
 +           Overrides serial consistency level (default level is SERIAL).
 +
 +        SERIAL CONSISTENCY <level>
 +
 +           Sets consistency level for future conditional updates.
 +
 +           Valid consistency levels:
 +
 +           SERIAL, LOCAL_SERIAL.
 +
 +        SERIAL CONSISTENCY
 +
 +           SERIAL CONSISTENCY with no arguments shows the current consistency level.
 +        """
 +        level = parsed.get_binding('level')
 +        if level is None:
 +            print 'Current serial consistency level is %s.' % (cassandra.ConsistencyLevel.value_to_name[self.serial_consistency_level])
 +            return
 +
 +        self.serial_consistency_level = cassandra.ConsistencyLevel.name_to_value[level.upper()]
 +        print 'Serial consistency level set to %s.' % (level.upper(),)
 +
 +    def do_login(self, parsed):
 +        """
 +        LOGIN [cqlsh only]
 +
 +           Changes login information without requiring restart.
 +
 +        LOGIN <username> (<password>)
 +
 +           Login using the specified username. If password is specified, it will be used
 +           otherwise, you will be prompted to enter.
 +        """
 +        username = parsed.get_binding('username')
 +        password = parsed.get_binding('password')
 +        if password is None:
 +            password = getpass.getpass()
 +        else:
 +            password = password[1:-1]
 +
 +        auth_provider = PlainTextAuthProvider(username=username, password=password)
 +
 +        conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=self.conn.cql_version,
 +                       protocol_version=self.conn.protocol_version,
 +                       auth_provider=auth_provider,
 +                       ssl_options=self.conn.ssl_options,
 +                       load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]),
 +                       control_connection_timeout=self.conn.connect_timeout,
 +                       connect_timeout=self.conn.connect_timeout)
 +
 +        if self.current_keyspace:
 +            session = conn.connect(self.current_keyspace)
 +        else:
 +            session = conn.connect()
 +
 +        # Update after we've connected in case we fail to authenticate
 +        self.conn = conn
 +        self.auth_provider = auth_provider
 +        self.username = username
 +        self.session = session
 +
 +    def do_exit(self, parsed=None):
 +        """
 +        EXIT/QUIT [cqlsh only]
 +
 +        Exits cqlsh.
 +        """
 +        self.stop = True
 +        if self.owns_connection:
 +            self.conn.shutdown()
 +    do_quit = do_exit
 +
 +    def do_clear(self, parsed):
 +        """
 +        CLEAR/CLS [cqlsh only]
 +
 +        Clears the console.
 +        """
 +        import subprocess
 +        subprocess.call(['clear', 'cls'][is_win], shell=True)
 +    do_cls = do_clear
 +
 +    def do_debug(self, parsed):
 +        import pdb
 +        pdb.set_trace()
 +
 +    def get_help_topics(self):
 +        topics = [t[3:] for t in dir(self) if t.startswith('do_') and getattr(self, t, None).__doc__]
 +        for hide_from_help in ('quit',):
 +            topics.remove(hide_from_help)
 +        return topi

<TRUNCATED>

[08/23] cassandra git commit: Merge commit 'c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc' into cassandra-2.2

Posted by sl...@apache.org.
Merge commit 'c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc' into cassandra-2.2


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b74ffeaf
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b74ffeaf
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b74ffeaf

Branch: refs/heads/trunk
Commit: b74ffeafd2e08a7669013d6b4ba87980e1ad5379
Parents: 6e0395e c3d2f26
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue Mar 8 10:22:22 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Mar 8 10:23:08 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                |    1 +
 bin/cqlsh.py               |   28 +-
 pylib/cqlshlib/copyutil.py | 1160 +++++++++++++++++++++++++--------------
 pylib/cqlshlib/util.py     |   35 +-
 pylib/setup.py             |    2 +
 5 files changed, 809 insertions(+), 417 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b74ffeaf/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 89fa904,d6b085c..7b67cdc
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,30 -1,5 +1,31 @@@
 -2.1.14
 +2.2.6
 + * Fix AE in nodetool cfstats (backport CASSANDRA-10859) (CASSANDRA-11297)
 + * Unresolved hostname leads to replace being ignored (CASSANDRA-11210)
 + * Fix filtering on non-primary key columns for thrift static column families
 +   (CASSANDRA-6377)
 + * Only log yaml config once, at startup (CASSANDRA-11217)
 + * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
 + * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
 + * Range.compareTo() violates the contract of Comparable (CASSANDRA-11216)
 + * Avoid NPE when serializing ErrorMessage with null message (CASSANDRA-11167)
 + * Replacing an aggregate with a new version doesn't reset INITCOND (CASSANDRA-10840)
 + * (cqlsh) cqlsh cannot be called through symlink (CASSANDRA-11037)
 + * fix ohc and java-driver pom dependencies in build.xml (CASSANDRA-10793)
 + * Protect from keyspace dropped during repair (CASSANDRA-11065)
 + * Handle adding fields to a UDT in SELECT JSON and toJson() (CASSANDRA-11146)
 + * Better error message for cleanup (CASSANDRA-10991)
 + * cqlsh pg-style-strings broken if line ends with ';' (CASSANDRA-11123)
 + * Use cloned TokenMetadata in size estimates to avoid race against membership check
 +   (CASSANDRA-10736)
 + * Always persist upsampled index summaries (CASSANDRA-10512)
 + * (cqlsh) Fix inconsistent auto-complete (CASSANDRA-10733)
 + * Make SELECT JSON and toJson() threadsafe (CASSANDRA-11048)
 + * Fix SELECT on tuple relations for mixed ASC/DESC clustering order (CASSANDRA-7281)
 + * (cqlsh) Support utf-8/cp65001 encoding on Windows (CASSANDRA-11030)
 + * Fix paging on DISTINCT queries repeats result when first row in partition changes
 +   (CASSANDRA-10010)
 +Merged from 2.1:
+  * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
   * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
   * Don't remove FailureDetector history on removeEndpoint (CASSANDRA-10371)
   * Only notify if repair status changed (CASSANDRA-11172)


[03/23] cassandra git commit: COPY FROM on large datasets: fix progress report and debug performance

Posted by sl...@apache.org.
COPY FROM on large datasets: fix progress report and debug performance

patch by Stefania Alborghetti; reviewed by Adam Holmberg for CASSANDRA-11053


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c3d2f26f
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c3d2f26f
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c3d2f26f

Branch: refs/heads/trunk
Commit: c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc
Parents: 0129f70
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Thu Jan 28 14:31:55 2016 +0800
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Mar 8 10:19:13 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                |    1 +
 bin/cqlsh                  |   28 +-
 pylib/cqlshlib/copyutil.py | 1160 +++++++++++++++++++++++++--------------
 pylib/cqlshlib/util.py     |   35 +-
 pylib/setup.py             |    2 +
 5 files changed, 809 insertions(+), 417 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index eed9035..d6b085c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.14
+ * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
  * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
  * Don't remove FailureDetector history on removeEndpoint (CASSANDRA-10371)
  * Only notify if repair status changed (CASSANDRA-11172)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/bin/cqlsh
----------------------------------------------------------------------
diff --git a/bin/cqlsh b/bin/cqlsh
index 7a39636..374e588 100755
--- a/bin/cqlsh
+++ b/bin/cqlsh
@@ -417,7 +417,7 @@ COPY_COMMON_OPTIONS = ['DELIMITER', 'QUOTE', 'ESCAPE', 'HEADER', 'NULL', 'DATETI
                        'MAXATTEMPTS', 'REPORTFREQUENCY', 'DECIMALSEP', 'THOUSANDSSEP', 'BOOLSTYLE',
                        'NUMPROCESSES', 'CONFIGFILE', 'RATEFILE']
 COPY_FROM_OPTIONS = ['CHUNKSIZE', 'INGESTRATE', 'MAXBATCHSIZE', 'MINBATCHSIZE', 'MAXROWS',
-                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE']
+                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE', 'PREPAREDSTATEMENTS']
 COPY_TO_OPTIONS = ['ENCODING', 'PAGESIZE', 'PAGETIMEOUT', 'BEGINTOKEN', 'ENDTOKEN', 'MAXOUTPUTSIZE', 'MAXREQUESTS']
 
 
@@ -533,8 +533,23 @@ def insert_driver_hooks():
 
 def extend_cql_deserialization():
     """
-    The python driver returns BLOBs as string, but we expect them as bytearrays
+    The python driver returns BLOBs as string, but we expect them as bytearrays; therefore we change
+    the implementation of cassandra.cqltypes.BytesType.deserialize.
+
+    The deserializers package exists only when the driver has been compiled with cython extensions and
+    cassandra.deserializers.DesBytesType replaces cassandra.cqltypes.BytesType.deserialize.
+
+    DesBytesTypeByteArray is a fast deserializer that converts blobs into bytearrays but it was
+    only introduced recently (3.1.0). If it is available we use it, otherwise we remove
+    cassandra.deserializers.DesBytesType so that we fall back onto cassandra.cqltypes.BytesType.deserialize
+    just like in the case where no cython extensions are present.
     """
+    if hasattr(cassandra, 'deserializers'):
+        if hasattr(cassandra.deserializers, 'DesBytesTypeByteArray'):
+            cassandra.deserializers.DesBytesType = cassandra.deserializers.DesBytesTypeByteArray
+        else:
+            del cassandra.deserializers.DesBytesType
+
     cassandra.cqltypes.BytesType.deserialize = staticmethod(lambda byts, protocol_version: bytearray(byts))
     cassandra.cqltypes.CassandraType.support_empty_values = True
 
@@ -1534,9 +1549,9 @@ class Shell(cmd.Cmd):
 
         Available COPY FROM options and defaults:
 
-          CHUNKSIZE=1000          - the size of chunks passed to worker processes
+          CHUNKSIZE=5000          - the size of chunks passed to worker processes
           INGESTRATE=100000       - an approximate ingest rate in rows per second
-          MINBATCHSIZE=2          - the minimum size of an import batch
+          MINBATCHSIZE=10         - the minimum size of an import batch
           MAXBATCHSIZE=20         - the maximum size of an import batch
           MAXROWS=-1              - the maximum number of rows, -1 means no maximum
           SKIPROWS=0              - the number of rows to skip
@@ -1545,6 +1560,11 @@ class Shell(cmd.Cmd):
           MAXINSERTERRORS=-1      - the maximum global number of insert errors, -1 means no maximum
           ERRFILE=''              - a file where to store all rows that could not be imported, by default this is
                                     import_ks_table.err where <ks> is your keyspace and <table> is your table name.
+          PREPAREDSTATEMENTS=True - whether to use prepared statements when importing, by default True. Set this to
+                                    False if you don't mind shifting data parsing to the cluster. The cluster will also
+                                    have to compile every batch statement. For large and oversized clusters
+                                    this will result in a faster import but for smaller clusters it may generate
+                                    timeouts.
 
         Available COPY TO options and defaults:
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/pylib/cqlshlib/copyutil.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/copyutil.py b/pylib/cqlshlib/copyutil.py
index f9e4a85..cd03765 100644
--- a/pylib/cqlshlib/copyutil.py
+++ b/pylib/cqlshlib/copyutil.py
@@ -1,3 +1,5 @@
+# cython: profile=True
+
 # Licensed to the Apache Software Foundation (ASF) under one
 # or more contributor license agreements.  See the NOTICE file
 # distributed with this work for additional information
@@ -21,25 +23,29 @@ import json
 import glob
 import multiprocessing as mp
 import os
-import Queue
+import platform
+import random
 import re
 import struct
 import sys
 import time
 import traceback
 
+from bisect import bisect_right
 from calendar import timegm
-from collections import defaultdict, deque, namedtuple
+from collections import defaultdict, namedtuple
 from decimal import Decimal
 from random import randrange
 from StringIO import StringIO
+from select import select
 from threading import Lock
 from uuid import UUID
+from util import profile_on, profile_off
 
 from cassandra.cluster import Cluster
 from cassandra.cqltypes import ReversedType, UserType
-from cassandra.metadata import protect_name, protect_names
-from cassandra.policies import RetryPolicy, WhiteListRoundRobinPolicy, TokenAwarePolicy, DCAwareRoundRobinPolicy
+from cassandra.metadata import protect_name, protect_names, protect_value
+from cassandra.policies import RetryPolicy, WhiteListRoundRobinPolicy, DCAwareRoundRobinPolicy
 from cassandra.query import BatchStatement, BatchType, SimpleStatement, tuple_factory
 from cassandra.util import Date, Time
 
@@ -48,6 +54,10 @@ from displaying import NO_COLOR_MAP
 from formatting import format_value_default, EMPTY, get_formatter
 from sslhandling import ssl_settings
 
+PROFILE_ON = False
+STRACE_ON = False
+IS_LINUX = platform.system() == 'Linux'
+
 CopyOptions = namedtuple('CopyOptions', 'copy dialect unrecognized')
 
 
@@ -59,6 +69,81 @@ def safe_normpath(fname):
     return os.path.normpath(os.path.expanduser(fname)) if fname else fname
 
 
+class OneWayChannel(object):
+    """
+    A one way pipe protected by two process level locks, one for reading and one for writing.
+    """
+    def __init__(self):
+        self.reader, self.writer = mp.Pipe(duplex=False)
+        self.rlock = mp.Lock()
+        self.wlock = mp.Lock()
+
+    def send(self, obj):
+        with self.wlock:
+            self.writer.send(obj)
+
+    def recv(self):
+        with self.rlock:
+            return self.reader.recv()
+
+    def close(self):
+        self.reader.close()
+        self.writer.close()
+
+
+class OneWayChannels(object):
+    """
+    A group of one way channels.
+    """
+    def __init__(self, num_channels):
+        self.channels = [OneWayChannel() for _ in xrange(num_channels)]
+        self._readers = [ch.reader for ch in self.channels]
+        self._rlocks = [ch.rlock for ch in self.channels]
+        self._rlocks_by_readers = dict([(ch.reader, ch.rlock) for ch in self.channels])
+        self.num_channels = num_channels
+
+        self.recv = self.recv_select if IS_LINUX else self.recv_polling
+
+    def recv_select(self, timeout):
+        """
+        Implementation of the recv method for Linux, where select is available. Receive an object from
+        all pipes that are ready for reading without blocking.
+        """
+        readable, _, _ = select(self._readers, [], [], timeout)
+        for r in readable:
+            with self._rlocks_by_readers[r]:
+                try:
+                    yield r.recv()
+                except EOFError:
+                    continue
+
+    def recv_polling(self, timeout):
+        """
+        Implementation of the recv method for platforms where select() is not available for pipes.
+        We poll on all of the readers with a very small timeout. We stop when the timeout specified
+        has been received but we may exceed it since we check all processes during each sweep.
+        """
+        start = time.time()
+        while True:
+            for i, r in enumerate(self._readers):
+                with self._rlocks[i]:
+                    if r.poll(0.000000001):
+                        try:
+                            yield r.recv()
+                        except EOFError:
+                            continue
+
+            if time.time() - start > timeout:
+                break
+
+    def close(self):
+        for ch in self.channels:
+            try:
+                ch.close()
+            except:
+                pass
+
+
 class CopyTask(object):
     """
     A base class for ImportTask and ExportTask
@@ -72,15 +157,18 @@ class CopyTask(object):
         self.protocol_version = protocol_version
         self.config_file = config_file
         # do not display messages when exporting to STDOUT
-        self.printmsg = self._printmsg if self.fname is not None or direction == 'in' else lambda _, eol='\n': None
+        self.printmsg = self._printmsg if self.fname is not None or direction == 'from' else lambda _, eol='\n': None
         self.options = self.parse_options(opts, direction)
 
         self.num_processes = self.options.copy['numprocesses']
+        if direction == 'in':
+            self.num_processes += 1  # add the feeder process
+
         self.printmsg('Using %d child processes' % (self.num_processes,))
 
         self.processes = []
-        self.inmsg = mp.Queue()
-        self.outmsg = mp.Queue()
+        self.inmsg = OneWayChannels(self.num_processes)
+        self.outmsg = OneWayChannels(self.num_processes)
 
         self.columns = CopyTask.get_columns(shell, ks, table, columns)
         self.time_start = time.time()
@@ -166,10 +254,10 @@ class CopyTask(object):
         copy_options['maxattempts'] = int(opts.pop('maxattempts', 5))
         copy_options['dtformats'] = opts.pop('datetimeformat', shell.display_time_format)
         copy_options['float_precision'] = shell.display_float_precision
-        copy_options['chunksize'] = int(opts.pop('chunksize', 1000))
+        copy_options['chunksize'] = int(opts.pop('chunksize', 5000))
         copy_options['ingestrate'] = int(opts.pop('ingestrate', 100000))
         copy_options['maxbatchsize'] = int(opts.pop('maxbatchsize', 20))
-        copy_options['minbatchsize'] = int(opts.pop('minbatchsize', 2))
+        copy_options['minbatchsize'] = int(opts.pop('minbatchsize', 10))
         copy_options['reportfrequency'] = float(opts.pop('reportfrequency', 0.25))
         copy_options['consistencylevel'] = shell.consistency_level
         copy_options['decimalsep'] = opts.pop('decimalsep', '.')
@@ -186,6 +274,7 @@ class CopyTask(object):
         copy_options['errfile'] = safe_normpath(opts.pop('errfile', 'import_%s_%s.err' % (self.ks, self.table,)))
         copy_options['ratefile'] = safe_normpath(opts.pop('ratefile', ''))
         copy_options['maxoutputsize'] = int(opts.pop('maxoutputsize', '-1'))
+        copy_options['preparedstatements'] = bool(opts.pop('preparedstatements', 'true').lower() == 'true')
 
         self.check_options(copy_options)
         return CopyOptions(copy=copy_options, dialect=dialect_options, unrecognized=opts)
@@ -206,14 +295,17 @@ class CopyTask(object):
     def get_num_processes(cap):
         """
         Pick a reasonable number of child processes. We need to leave at
-        least one core for the parent process.  This doesn't necessarily
-        need to be capped, but 4 is currently enough to keep
-        a single local Cassandra node busy so we use this for import, whilst
-        for export we use 16 since we can connect to multiple Cassandra nodes.
-        Eventually this parameter will become an option.
+        least one core for the parent process.
+        """
+        return max(1, min(cap, CopyTask.get_num_cores() - 1))
+
+    @staticmethod
+    def get_num_cores():
+        """
+        Return the number of cores if available.
         """
         try:
-            return max(1, min(cap, mp.cpu_count() - 1))
+            return mp.cpu_count()
         except NotImplementedError:
             return 1
 
@@ -244,28 +336,40 @@ class CopyTask(object):
         return shell.get_column_names(ks, table) if not columns else columns
 
     def close(self):
-        for process in self.processes:
-            process.terminate()
-
+        self.stop_processes()
         self.inmsg.close()
         self.outmsg.close()
 
     def num_live_processes(self):
         return sum(1 for p in self.processes if p.is_alive())
 
+    @staticmethod
+    def get_pid():
+        return os.getpid() if hasattr(os, 'getpid') else None
+
+    @staticmethod
+    def trace_process(pid):
+        if pid and STRACE_ON:
+            os.system("strace -vvvv -c -o strace.{pid}.out -e trace=all -p {pid}&".format(pid=pid))
+
+    def start_processes(self):
+        for i, process in enumerate(self.processes):
+            process.start()
+            self.trace_process(process.pid)
+
+        self.trace_process(self.get_pid())
+
+    def stop_processes(self):
+        for process in self.processes:
+            process.terminate()
+
     def make_params(self):
         """
         Return a dictionary of parameters to be used by the worker processes.
         On Windows this dictionary must be pickle-able.
-
-        inmsg is the message queue flowing from parent to child process, so outmsg from the parent point
-        of view and, vice-versa,  outmsg is the message queue flowing from child to parent, so inmsg
-        from the parent point of view, hence the two are swapped below.
         """
         shell = self.shell
-        return dict(inmsg=self.outmsg,  # see comment above
-                    outmsg=self.inmsg,  # see comment above
-                    ks=self.ks,
+        return dict(ks=self.ks,
                     table=self.table,
                     local_dc=self.local_dc,
                     columns=self.columns,
@@ -281,6 +385,17 @@ class CopyTask(object):
                     debug=shell.debug
                     )
 
+    def update_params(self, params, i):
+        """
+        Add the communication channels to the parameters to be passed to the worker process:
+            inmsg is the message queue flowing from parent to child process, so outmsg from the parent point
+            of view and, vice-versa,  outmsg is the message queue flowing from child to parent, so inmsg
+            from the parent point of view, hence the two are swapped below.
+        """
+        params['inmsg'] = self.outmsg.channels[i]
+        params['outmsg'] = self.inmsg.channels[i]
+        return params
+
 
 class ExportWriter(object):
     """
@@ -414,10 +529,9 @@ class ExportTask(CopyTask):
 
         params = self.make_params()
         for i in xrange(self.num_processes):
-            self.processes.append(ExportProcess(params))
+            self.processes.append(ExportProcess(self.update_params(params, i)))
 
-        for process in self.processes:
-            process.start()
+        self.start_processes()
 
         try:
             self.export_records(ranges)
@@ -468,11 +582,12 @@ class ExportTask(CopyTask):
 
             return ret
 
-        def make_range_data(replicas=[]):
+        def make_range_data(replicas=None):
             hosts = []
-            for r in replicas:
-                if r.is_up and r.datacenter == local_dc:
-                    hosts.append(r.address)
+            if replicas:
+                for r in replicas:
+                    if r.is_up and r.datacenter == local_dc:
+                        hosts.append(r.address)
             if not hosts:
                 hosts.append(hostname)  # fallback to default host if no replicas in current dc
             return {'hosts': tuple(hosts), 'attempts': 0, 'rows': 0}
@@ -542,10 +657,13 @@ class ExportTask(CopyTask):
             return None
 
     def send_work(self, ranges, tokens_to_send):
+        i = 0
         for token_range in tokens_to_send:
-            self.outmsg.put((token_range, ranges[token_range]))
+            self.outmsg.channels[i].send((token_range, ranges[token_range]))
             ranges[token_range]['attempts'] += 1
 
+            i = i + 1 if i < self.num_processes - 1 else 0
+
     def export_records(self, ranges):
         """
         Send records to child processes and monitor them by collecting their results
@@ -568,8 +686,7 @@ class ExportTask(CopyTask):
         succeeded = 0
         failed = 0
         while (failed + succeeded) < total_requests and self.num_live_processes() == num_processes:
-            try:
-                token_range, result = self.inmsg.get(timeout=1.0)
+            for token_range, result in self.inmsg.recv(timeout=0.1):
                 if token_range is None and result is None:  # a request has finished
                     succeeded += 1
                 elif isinstance(result, Exception):  # an error occurred
@@ -594,8 +711,6 @@ class ExportTask(CopyTask):
                     self.writer.write(data, num)
                     meter.increment(n=num)
                     ranges[token_range]['rows'] += num
-            except Queue.Empty:
-                pass
 
         if self.num_live_processes() < len(processes):
             for process in processes:
@@ -612,7 +727,7 @@ class ExportTask(CopyTask):
                        self.describe_interval(time.time() - self.time_start)))
 
 
-class ImportReader(object):
+class FilesReader(object):
     """
     A wrapper around a csv reader to keep track of when we have
     exhausted reading input files. We are passed a comma separated
@@ -620,18 +735,15 @@ class ImportReader(object):
     We generate a source generator and we read each source one
     by one.
     """
-    def __init__(self, task):
-        self.shell = task.shell
-        self.options = task.options
-        self.printmsg = task.printmsg
-        self.chunk_size = self.options.copy['chunksize']
-        self.header = self.options.copy['header']
-        self.max_rows = self.options.copy['maxrows']
-        self.skip_rows = self.options.copy['skiprows']
-        self.sources = self.get_source(task.fname)
+    def __init__(self, fname, options):
+        self.chunk_size = options.copy['chunksize']
+        self.header = options.copy['header']
+        self.max_rows = options.copy['maxrows']
+        self.skip_rows = options.copy['skiprows']
+        self.fname = fname
+        self.sources = None  # must be created later due to pickle problems on Windows
         self.num_sources = 0
         self.current_source = None
-        self.current_reader = None
         self.num_read = 0
 
     def get_source(self, paths):
@@ -640,35 +752,33 @@ class ImportReader(object):
          wrapping the source input, file name and a boolean indicating
          if it requires closing.
         """
-        shell = self.shell
-        LineSource = namedtuple('LineSource', 'input close fname')
-
         def make_source(fname):
             try:
-                ret = LineSource(input=open(fname, 'rb'), close=True, fname=fname)
-                return ret
+                return open(fname, 'rb')
             except IOError, e:
-                shell.printerr("Can't open %r for reading: %s" % (fname, e))
+                self.printmsg("Can't open %r for reading: %s" % (fname, e))
                 return None
 
-        if paths is None:
-            self.printmsg("[Use \. on a line by itself to end input]")
-            yield LineSource(input=shell.use_stdin_reader(prompt='[copy] ', until=r'\.'), close=False, fname='')
-        else:
-            for path in paths.split(','):
-                path = path.strip()
-                if os.path.isfile(path):
-                    yield make_source(path)
-                else:
-                    for f in glob.glob(path):
-                        yield (make_source(f))
+        for path in paths.split(','):
+            path = path.strip()
+            if os.path.isfile(path):
+                yield make_source(path)
+            else:
+                for f in glob.glob(path):
+                    yield (make_source(f))
+
+    @staticmethod
+    def printmsg(msg, eol='\n'):
+        sys.stdout.write(msg + eol)
+        sys.stdout.flush()
 
     def start(self):
+        self.sources = self.get_source(self.fname)
         self.next_source()
 
     @property
     def exhausted(self):
-        return not self.current_reader
+        return not self.current_source
 
     def next_source(self):
         """
@@ -679,40 +789,34 @@ class ImportReader(object):
         while self.current_source is None:
             try:
                 self.current_source = self.sources.next()
-                if self.current_source and self.current_source.fname:
+                if self.current_source:
                     self.num_sources += 1
             except StopIteration:
                 return False
 
         if self.header:
-            self.current_source.input.next()
+            self.current_source.next()
 
-        self.current_reader = csv.reader(self.current_source.input, **self.options.dialect)
         return True
 
     def close_current_source(self):
         if not self.current_source:
             return
 
-        if self.current_source.close:
-            self.current_source.input.close()
-        elif self.shell.tty:
-            print
-
+        self.current_source.close()
         self.current_source = None
-        self.current_reader = None
 
     def close(self):
         self.close_current_source()
 
     def read_rows(self, max_rows):
-        if not self.current_reader:
+        if not self.current_source:
             return []
 
         rows = []
         for i in xrange(min(max_rows, self.chunk_size)):
             try:
-                row = self.current_reader.next()
+                row = self.current_source.next()
                 self.num_read += 1
 
                 if 0 <= self.max_rows < self.num_read:
@@ -729,13 +833,91 @@ class ImportReader(object):
         return filter(None, rows)
 
 
-class ImportErrors(object):
+class PipeReader(object):
     """
-    A small class for managing import errors
+    A class for reading rows received on a pipe, this is used for reading input from STDIN
+    """
+    def __init__(self, inmsg, options):
+        self.inmsg = inmsg
+        self.chunk_size = options.copy['chunksize']
+        self.header = options.copy['header']
+        self.max_rows = options.copy['maxrows']
+        self.skip_rows = options.copy['skiprows']
+        self.num_read = 0
+        self.exhausted = False
+        self.num_sources = 1
+
+    def start(self):
+        pass
+
+    def read_rows(self, max_rows):
+        rows = []
+        for i in xrange(min(max_rows, self.chunk_size)):
+            row = self.inmsg.recv()
+            if row is None:
+                self.exhausted = True
+                break
+
+            self.num_read += 1
+            if 0 <= self.max_rows < self.num_read:
+                self.exhausted = True
+                break  # max rows exceeded
+
+            if self.header or self.num_read < self.skip_rows:
+                self.header = False  # skip header or initial skip_rows rows
+                continue
+
+            rows.append(row)
+
+        return rows
+
+
+class ImportProcessResult(object):
+    """
+    An object sent from ImportProcess instances to the parent import task in order to indicate progress.
+    """
+    def __init__(self, imported=0):
+        self.imported = imported
+
+
+class FeedingProcessResult(object):
+    """
+    An object sent from FeedingProcess instances to the parent import task in order to indicate progress.
+    """
+    def __init__(self, sent, reader):
+        self.sent = sent
+        self.num_sources = reader.num_sources
+        self.skip_rows = reader.skip_rows
+
+
+class ImportTaskError(object):
+    """
+    An object sent from child processes (feeder or workers) to the parent import task to indicate an error.
+    """
+    def __init__(self, name, msg, rows=None, attempts=1, final=True):
+        self.name = name
+        self.msg = msg
+        self.rows = rows if rows else []
+        self.attempts = attempts
+        self.final = final
+
+    def is_parse_error(self):
+        """
+        We treat read and parse errors as unrecoverable and we have different global counters for giving up when
+        a maximum has been reached. We consider value and type errors as parse errors as well since they
+        are typically non recoverable.
+        """
+        name = self.name
+        return name.startswith('ValueError') or name.startswith('TypeError') or \
+            name.startswith('ParseError') or name.startswith('IndexError') or name.startswith('ReadError')
+
+
+class ImportErrorHandler(object):
+    """
+    A class for managing import errors
     """
     def __init__(self, task):
         self.shell = task.shell
-        self.reader = task.reader
         self.options = task.options
         self.printmsg = task.printmsg
         self.max_attempts = self.options.copy['maxattempts']
@@ -771,42 +953,26 @@ class ImportErrors(object):
             for row in rows:
                 writer.writerow(row)
 
-    def handle_error(self, err, batch):
+    def handle_error(self, err):
         """
         Handle an error by printing the appropriate error message and incrementing the correct counter.
-        Return true if we should retry this batch, false if the error is non-recoverable
         """
         shell = self.shell
-        err = str(err)
 
-        if self.is_parse_error(err):
-            self.parse_errors += len(batch['rows'])
-            self.add_failed_rows(batch['rows'])
-            shell.printerr("Failed to import %d rows: %s -  given up without retries"
-                           % (len(batch['rows']), err))
-            return False
+        if err.is_parse_error():
+            self.parse_errors += len(err.rows)
+            self.add_failed_rows(err.rows)
+            shell.printerr("Failed to import %d rows: %s - %s,  given up without retries"
+                           % (len(err.rows), err.name, err.msg))
         else:
-            self.insert_errors += len(batch['rows'])
-            if batch['attempts'] < self.max_attempts:
-                shell.printerr("Failed to import %d rows: %s -  will retry later, attempt %d of %d"
-                               % (len(batch['rows']), err, batch['attempts'],
-                                  self.max_attempts))
-                return True
+            self.insert_errors += len(err.rows)
+            if not err.final:
+                shell.printerr("Failed to import %d rows: %s - %s,  will retry later, attempt %d of %d"
+                               % (len(err.rows), err.name, err.msg, err.attempts, self.max_attempts))
             else:
-                self.add_failed_rows(batch['rows'])
-                shell.printerr("Failed to import %d rows: %s -  given up after %d attempts"
-                               % (len(batch['rows']), err, batch['attempts']))
-                return False
-
-    @staticmethod
-    def is_parse_error(err):
-        """
-        We treat parse errors as unrecoverable and we have different global counters for giving up when
-        a maximum has been reached. We consider value and type errors as parse errors as well since they
-        are typically non recoverable.
-        """
-        return err.startswith('ValueError') or err.startswith('TypeError') or \
-            err.startswith('ParseError') or err.startswith('IndexError')
+                self.add_failed_rows(err.rows)
+                shell.printerr("Failed to import %d rows: %s - %s,  given up after %d attempts"
+                               % (len(err.rows), err.name, err.msg, err.attempts))
 
 
 class ImportTask(CopyTask):
@@ -818,22 +984,14 @@ class ImportTask(CopyTask):
         CopyTask.__init__(self, shell, ks, table, columns, fname, opts, protocol_version, config_file, 'from')
 
         options = self.options
-        self.ingest_rate = options.copy['ingestrate']
-        self.max_attempts = options.copy['maxattempts']
-        self.header = options.copy['header']
         self.skip_columns = [c.strip() for c in self.options.copy['skipcols'].split(',')]
         self.valid_columns = [c for c in self.columns if c not in self.skip_columns]
         self.table_meta = self.shell.get_table_meta(self.ks, self.table)
-        self.batch_id = 0
         self.receive_meter = RateMeter(log_fcn=self.printmsg,
                                        update_interval=options.copy['reportfrequency'],
                                        log_file=options.copy['ratefile'])
-        self.send_meter = RateMeter(log_fcn=None, update_interval=1)
-        self.reader = ImportReader(self)
-        self.import_errors = ImportErrors(self)
-        self.retries = deque([])
-        self.failed = 0
-        self.succeeded = 0
+        self.error_handler = ImportErrorHandler(self)
+        self.feeding_result = None
         self.sent = 0
 
     def make_params(self):
@@ -861,17 +1019,24 @@ class ImportTask(CopyTask):
         self.printmsg("\nStarting copy of %s.%s with columns %s." % (self.ks, self.table, self.valid_columns))
 
         try:
-            self.reader.start()
             params = self.make_params()
 
-            for i in range(self.num_processes):
-                self.processes.append(ImportProcess(params))
+            for i in range(self.num_processes - 1):
+                self.processes.append(ImportProcess(self.update_params(params, i)))
+
+            feeder = FeedingProcess(self.outmsg.channels[-1], self.inmsg.channels[-1],
+                                    self.outmsg.channels[:-1], self.fname, self.options)
+            self.processes.append(feeder)
 
-            for process in self.processes:
-                process.start()
+            self.start_processes()
+
+            pr = profile_on() if PROFILE_ON else None
 
             self.import_records()
 
+            if pr:
+                profile_off(pr, file_name='parent_profile_%d.txt' % (os.getpid(),))
+
         except Exception, exc:
             shell.printerr(str(exc))
             if shell.debug:
@@ -880,9 +1045,22 @@ class ImportTask(CopyTask):
         finally:
             self.close()
 
-    def close(self):
-        CopyTask.close(self)
-        self.reader.close()
+    def send_stdin_rows(self):
+        """
+        We need to pass stdin rows to the feeder process as it is not safe to pickle or share stdin
+        directly (in case of file the child process would close it). This is a very primitive support
+        for STDIN import in that we we won't start reporting progress until STDIN is fully consumed. I
+        think this is reasonable.
+        """
+        shell = self.shell
+
+        self.printmsg("[Use \. on a line by itself to end input]")
+        for row in shell.use_stdin_reader(prompt='[copy] ', until=r'.'):
+            self.outmsg.channels[-1].send(row)
+
+        self.outmsg.channels[-1].send(None)
+        if shell.tty:
+            print
 
     def import_records(self):
         """
@@ -890,114 +1068,137 @@ class ImportTask(CopyTask):
         Send data (batches or retries) up to the max ingest rate. If we are waiting for stuff to
         receive check the incoming queue.
         """
-        reader = self.reader
-
-        while self.has_more_to_send(reader) or self.has_more_to_receive():
-            if self.has_more_to_send(reader):
-                self.send_batches(reader)
+        if not self.fname:
+            self.send_stdin_rows()
 
-            if self.has_more_to_receive():
-                self.receive()
+        while self.feeding_result is None or self.receive_meter.total_records < self.feeding_result.sent:
+            self.receive_results()
 
-            if self.import_errors.max_exceeded() or not self.all_processes_running():
+            if self.error_handler.max_exceeded() or not self.all_processes_running():
                 break
 
-        if self.import_errors.num_rows_failed:
+        if self.error_handler.num_rows_failed:
             self.shell.printerr("Failed to process %d rows; failed rows written to %s" %
-                                (self.import_errors.num_rows_failed,
-                                 self.import_errors.err_file))
+                                (self.error_handler.num_rows_failed,
+                                 self.error_handler.err_file))
 
         if not self.all_processes_running():
             self.shell.printerr("{} child process(es) died unexpectedly, aborting"
                                 .format(self.num_processes - self.num_live_processes()))
+        else:
+            # it is only safe to write to processes if they are all running because the feeder process
+            # at the moment hangs whilst sending messages to a crashed worker process; in future
+            # we could do something about this by using a BoundedSemaphore to keep track of how many messages are
+            # queued on a pipe
+            for i, _ in enumerate(self.processes):
+                self.outmsg.channels[i].send(None)
+
+            if PROFILE_ON:
+                # allow time for worker processes to write profile results (only works if processes received
+                # the poison pill above)
+                time.sleep(5)
 
         self.printmsg("\n%d rows imported from %d files in %s (%d skipped)." %
                       (self.receive_meter.get_total_records(),
-                       self.reader.num_sources,
+                       self.feeding_result.num_sources if self.feeding_result else 0,
                        self.describe_interval(time.time() - self.time_start),
-                       self.reader.skip_rows))
-
-    def has_more_to_receive(self):
-        return (self.succeeded + self.failed) < self.sent
-
-    def has_more_to_send(self, reader):
-        return (not reader.exhausted) or self.retries
+                       self.feeding_result.skip_rows if self.feeding_result else 0))
 
     def all_processes_running(self):
-        return self.num_live_processes() == self.num_processes
+        return self.num_live_processes() == len(self.processes)
 
-    def receive(self):
-        start_time = time.time()
+    def receive_results(self):
+        """
+        Receive results from the worker processes, which will send the number of rows imported
+        or from the feeder process, which will send the number of rows sent when it has finished sending rows.
+        """
+        aggregate_result = ImportProcessResult()
+        try:
+            for result in self.inmsg.recv(timeout=0.1):
+                if isinstance(result, ImportProcessResult):
+                    aggregate_result.imported += result.imported
+                elif isinstance(result, ImportTaskError):
+                    self.error_handler.handle_error(result)
+                elif isinstance(result, FeedingProcessResult):
+                    self.feeding_result = result
+                else:
+                    raise ValueError("Unexpected result: %s" % (result,))
+        finally:
+            self.receive_meter.increment(aggregate_result.imported)
 
-        while time.time() - start_time < 0.001:
-            try:
-                batch, err = self.inmsg.get(timeout=0.00001)
 
-                if err is None:
-                    self.succeeded += batch['imported']
-                    self.receive_meter.increment(batch['imported'])
-                else:
-                    err = str(err)
+class FeedingProcess(mp.Process):
+    """
+    A process that reads from import sources and sends chunks to worker processes.
+    """
+    def __init__(self, inmsg, outmsg, worker_channels, fname, options):
+        mp.Process.__init__(self, target=self.run)
+        self.inmsg = inmsg
+        self.outmsg = outmsg
+        self.worker_channels = worker_channels
+        self.reader = FilesReader(fname, options) if fname else PipeReader(inmsg, options)
+        self.send_meter = RateMeter(log_fcn=None, update_interval=1)
+        self.ingest_rate = options.copy['ingestrate']
+        self.num_worker_processes = options.copy['numprocesses']
+        self.chunk_id = 0
+
+    def run(self):
+        pr = profile_on() if PROFILE_ON else None
 
-                    if self.import_errors.handle_error(err, batch):
-                        self.retries.append(self.reset_batch(batch))
-                    else:
-                        self.failed += len(batch['rows'])
+        self.inner_run()
 
-            except Queue.Empty:
-                pass
+        if pr:
+            profile_off(pr, file_name='feeder_profile_%d.txt' % (os.getpid(),))
 
-    def send_batches(self, reader):
+    def inner_run(self):
         """
         Send one batch per worker process to the queue unless we have exceeded the ingest rate.
         In the export case we queue everything and let the worker processes throttle using max_requests,
-        here we throttle using the ingest rate in the parent process because of memory usage concerns.
-
-        When we have finished reading the csv file, then send any retries.
+        here we throttle using the ingest rate in the feeding process because of memory usage concerns.
+        When finished we send back to the parent process the total number of rows sent.
         """
-        for _ in xrange(self.num_processes):
-            max_rows = self.ingest_rate - self.send_meter.current_record
-            if max_rows <= 0:
-                self.send_meter.maybe_update()
-                break
+        reader = self.reader
+        reader.start()
+        channels = self.worker_channels
+        sent = 0
+
+        while not reader.exhausted:
+            for ch in channels:
+                try:
+                    max_rows = self.ingest_rate - self.send_meter.current_record
+                    if max_rows <= 0:
+                        self.send_meter.maybe_update(sleep=False)
+                        continue
+
+                    rows = reader.read_rows(max_rows)
+                    if rows:
+                        sent += self.send_chunk(ch, rows)
+                except Exception, exc:
+                    self.outmsg.send(ImportTaskError(exc.__class__.__name__, exc.message))
+
+                if reader.exhausted:
+                    break
 
-            if not reader.exhausted:
-                rows = reader.read_rows(max_rows)
-                if rows:
-                    self.sent += self.send_batch(self.new_batch(rows))
-            elif self.retries:
-                batch = self.retries.popleft()
-                if len(batch['rows']) <= max_rows:
-                    self.send_batch(batch)
-                else:
-                    self.send_batch(self.split_batch(batch, batch['rows'][:max_rows]))
-                    self.retries.append(self.split_batch(batch, batch['rows'][max_rows:]))
-            else:
-                break
+        # send back to the parent process the number of rows sent to the worker processes
+        self.outmsg.send(FeedingProcessResult(sent, reader))
+
+        # wait for poison pill (None)
+        self.inmsg.recv()
 
-    def send_batch(self, batch):
-        batch['attempts'] += 1
-        num_rows = len(batch['rows'])
+    def send_chunk(self, ch, rows):
+        self.chunk_id += 1
+        num_rows = len(rows)
         self.send_meter.increment(num_rows)
-        self.outmsg.put(batch)
+        ch.send({'id': self.chunk_id, 'rows': rows, 'imported': 0, 'num_rows_sent': num_rows})
         return num_rows
 
-    def new_batch(self, rows):
-        self.batch_id += 1
-        return self.make_batch(self.batch_id, rows, 0)
-
-    @staticmethod
-    def reset_batch(batch):
-        batch['imported'] = 0
-        return batch
-
-    @staticmethod
-    def split_batch(batch, rows):
-        return ImportTask.make_batch(batch['id'], rows, batch['attempts'])
+    def close(self):
+        self.reader.close()
+        self.inmsg.close()
+        self.outmsg.close()
 
-    @staticmethod
-    def make_batch(batch_id, rows, attempts):
-        return {'id': batch_id, 'rows': rows, 'attempts': attempts, 'imported': 0}
+        for ch in self.worker_channels:
+            ch.close()
 
 
 class ChildProcess(mp.Process):
@@ -1029,6 +1230,7 @@ class ChildProcess(mp.Process):
         self.decimal_sep = options.copy['decimalsep']
         self.thousands_sep = options.copy['thousandssep']
         self.boolean_styles = options.copy['boolstyle']
+        self.max_attempts = options.copy['maxattempts']
         # Here we inject some failures for testing purposes, only if this environment variable is set
         if os.environ.get('CQLSH_COPY_TEST_FAILURES', ''):
             self.test_failures = json.loads(os.environ.get('CQLSH_COPY_TEST_FAILURES', ''))
@@ -1144,7 +1346,6 @@ class ExportProcess(ChildProcess):
         self.encoding = options.copy['encoding']
         self.float_precision = options.copy['float_precision']
         self.nullval = options.copy['nullval']
-        self.max_attempts = options.copy['maxattempts']
         self.max_requests = options.copy['maxrequests']
 
         self.hosts_to_sessions = dict()
@@ -1172,7 +1373,7 @@ class ExportProcess(ChildProcess):
                 time.sleep(0.001)  # 1 millisecond
                 continue
 
-            token_range, info = self.inmsg.get()
+            token_range, info = self.inmsg.recv()
             self.start_request(token_range, info)
 
     @staticmethod
@@ -1190,7 +1391,7 @@ class ExportProcess(ChildProcess):
     def report_error(self, err, token_range=None):
         msg = self.get_error_message(err, print_traceback=self.debug)
         self.printdebugmsg(msg)
-        self.outmsg.put((token_range, Exception(msg)))
+        self.outmsg.send((token_range, Exception(msg)))
 
     def start_request(self, token_range, info):
         """
@@ -1253,7 +1454,8 @@ class ExportProcess(ChildProcess):
             default_retry_policy=ExpBackoffRetryPolicy(self),
             compression=None,
             control_connection_timeout=self.connect_timeout,
-            connect_timeout=self.connect_timeout)
+            connect_timeout=self.connect_timeout,
+            idle_heartbeat_interval=0)
         session = ExportSession(new_cluster, self)
         self.hosts_to_sessions[host] = session
         return session
@@ -1265,7 +1467,7 @@ class ExportProcess(ChildProcess):
                 self.write_rows_to_csv(token_range, rows)
             else:
                 self.write_rows_to_csv(token_range, rows)
-                self.outmsg.put((None, None))
+                self.outmsg.send((None, None))
                 session.complete_request()
 
         def err_callback(err):
@@ -1286,7 +1488,7 @@ class ExportProcess(ChildProcess):
                 writer.writerow(map(self.format_value, row))
 
             data = (output.getvalue(), len(rows))
-            self.outmsg.put((token_range, data))
+            self.outmsg.send((token_range, data))
             output.close()
 
         except Exception, e:
@@ -1376,7 +1578,7 @@ class ImportConversion(object):
     A class for converting strings to values when importing from csv, used by ImportProcess,
     the parent.
     """
-    def __init__(self, parent, table_meta, statement):
+    def __init__(self, parent, table_meta, statement=None):
         self.ks = parent.ks
         self.table = parent.table
         self.columns = parent.valid_columns
@@ -1391,9 +1593,37 @@ class ImportConversion(object):
         self.primary_key_indexes = [self.columns.index(col.name) for col in self.table_meta.primary_key]
         self.partition_key_indexes = [self.columns.index(col.name) for col in self.table_meta.partition_key]
 
+        if statement is None:
+            self.use_prepared_statements = False
+            statement = self._get_primary_key_statement(parent, table_meta)
+        else:
+            self.use_prepared_statements = True
+
         self.proto_version = statement.protocol_version
-        self.cqltypes = dict([(c.name, c.type) for c in statement.column_metadata])
-        self.converters = dict([(c.name, self._get_converter(c.type)) for c in statement.column_metadata])
+
+        # the cql types and converters for the prepared statement, either the full statement or only the primary keys
+        self.cqltypes = [c.type for c in statement.column_metadata]
+        self.converters = [self._get_converter(c.type) for c in statement.column_metadata]
+
+        # the cql types for the entire statement, these are the same as the types above but
+        # only when using prepared statements
+        self.coltypes = [table_meta.columns[name].typestring for name in parent.valid_columns]
+        # these functions are used for non-prepared statements to protect values with quotes if required
+        self.protectors = [protect_value if t in ('ascii', 'text', 'timestamp', 'date', 'time', 'inet') else lambda v: v
+                           for t in self.coltypes]
+
+    @staticmethod
+    def _get_primary_key_statement(parent, table_meta):
+        """
+        We prepare a query statement to find out the types of the partition key columns so we can
+        route the update query to the correct replicas. As far as I understood this is the easiest
+        way to find out the types of the partition columns, we will never use this prepared statement
+        """
+        where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
+        select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(parent.ks),
+                                                         protect_name(parent.table),
+                                                         where_clause)
+        return parent.session.prepare(select_query)
 
     def _get_converter(self, cql_type):
         """
@@ -1581,27 +1811,25 @@ class ImportConversion(object):
 
         return converters.get(cql_type.typename, convert_unknown)
 
-    def get_row_values(self, row):
+    def convert_row(self, row):
         """
-        Parse the row into a list of row values to be returned
+        Convert the row into a list of parsed values if using prepared statements, else simply apply the
+        protection functions to escape values with quotes when required. Also check on the row length and
+        make sure primary partition key values aren't missing.
         """
-        def convert(n, val):
-            try:
-                return self.converters[self.columns[n]](val)
-            except Exception, e:
-                raise ParseError(e.message)
+        converters = self.converters if self.use_prepared_statements else self.protectors
 
-        ret = [None] * len(row)
-        for i, val in enumerate(row):
-            if val != self.nullval:
-                ret[i] = convert(i, val)
-            else:
-                if i in self.primary_key_indexes:
-                    raise ParseError(self.get_null_primary_key_message(i))
+        if len(row) != len(converters):
+            raise ParseError('Invalid row length %d should be %d' % (len(row), len(converters)))
 
-                ret[i] = None
+        for i in self.primary_key_indexes:
+            if row[i] == self.nullval:
+                raise ParseError(self.get_null_primary_key_message(i))
 
-        return ret
+        try:
+            return [conv(val) for conv, val in zip(converters, row)]
+        except Exception, e:
+            raise ParseError(e.message)
 
     def get_null_primary_key_message(self, idx):
         message = "Cannot insert null value for primary key column '%s'." % (self.columns[idx],)
@@ -1610,31 +1838,111 @@ class ImportConversion(object):
                        " the WITH NULL=<marker> option for COPY."
         return message
 
-    def get_row_partition_key_values(self, row):
+    def get_row_partition_key_values_fcn(self):
         """
-        Return a string composed of the partition key values, serialized and binary packed -
-        as expected by metadata.get_replicas(), see also BoundStatement.routing_key.
+        Return a function to convert a row into a string composed of the partition key values serialized
+        and binary packed (the tokens on the ring). Depending on whether we are using prepared statements, we
+        may have to convert the primary key values first, so we have two different serialize_value implementations.
+        We also return different functions depending on how many partition key indexes we have (single or multiple).
+        See also BoundStatement.routing_key.
         """
-        def serialize(n):
-            try:
-                c, v = self.columns[n], row[n]
-                if v == self.nullval:
-                    raise ParseError(self.get_null_primary_key_message(n))
-                return self.cqltypes[c].serialize(self.converters[c](v), self.proto_version)
-            except Exception, e:
-                raise ParseError(e.message)
+        def serialize_value_prepared(n, v):
+            return self.cqltypes[n].serialize(v, self.proto_version)
+
+        def serialize_value_not_prepared(n, v):
+            return self.cqltypes[n].serialize(self.converters[n](v), self.proto_version)
 
         partition_key_indexes = self.partition_key_indexes
-        if len(partition_key_indexes) == 1:
-            return serialize(partition_key_indexes[0])
-        else:
+        serialize = serialize_value_prepared if self.use_prepared_statements else serialize_value_not_prepared
+
+        def serialize_row_single(row):
+            return serialize(partition_key_indexes[0], row[partition_key_indexes[0]])
+
+        def serialize_row_multiple(row):
             pk_values = []
             for i in partition_key_indexes:
-                val = serialize(i)
+                val = serialize(i, row[i])
                 l = len(val)
                 pk_values.append(struct.pack(">H%dsB" % l, l, val, 0))
             return b"".join(pk_values)
 
+        if len(partition_key_indexes) == 1:
+            return serialize_row_single
+        return serialize_row_multiple
+
+
+class TokenMap(object):
+    """
+    A wrapper around the metadata token map to speed things up by caching ring token *values* and
+    replicas. It is very important that we use the token values, which are primitive types, rather
+    than the tokens classes when calling bisect_right() in split_batches(). If we use primitive values,
+    the bisect is done in compiled code whilst with token classes each comparison requires a call
+    into the interpreter to perform the cmp operation defined in Python. A simple test with 1 million bisect
+    operations on an array of 2048 tokens was done in 0.37 seconds with primitives and 2.25 seconds with
+    token classes. This is significant for large datasets because we need to do a bisect for each single row,
+    and if VNODES are used, the size of the token map can get quite large too.
+    """
+    def __init__(self, ks, hostname, local_dc, session):
+
+        self.ks = ks
+        self.hostname = hostname
+        self.local_dc = local_dc
+        self.metadata = session.cluster.metadata
+
+        self._initialize_ring()
+
+        # Note that refresh metadata is disabled by default and we currenlty do not intercept it
+        # If hosts are added, removed or moved during a COPY operation our token map is no longer optimal
+        # However we can cope with hosts going down and up since we filter for replicas that are up when
+        # making each batch
+
+    def _initialize_ring(self):
+        token_map = self.metadata.token_map
+        if token_map is None:
+            self.ring = [0]
+            self.replicas = [(self.metadata.get_host(self.hostname),)]
+            self.pk_to_token_value = lambda pk: 0
+            return
+
+        token_map.rebuild_keyspace(self.ks, build_if_absent=True)
+        tokens_to_hosts = token_map.tokens_to_hosts_by_ks.get(self.ks, None)
+        from_key = token_map.token_class.from_key
+
+        self.ring = [token.value for token in token_map.ring]
+        self.replicas = [tuple(tokens_to_hosts[token]) for token in token_map.ring]
+        self.pk_to_token_value = lambda pk: from_key(pk).value
+
+    @staticmethod
+    def get_ring_pos(ring, val):
+        idx = bisect_right(ring, val)
+        return idx if idx < len(ring) else 0
+
+    def filter_replicas(self, hosts):
+        shuffled = tuple(sorted(hosts, key=lambda k: random.random()))
+        return filter(lambda r: r.is_up and r.datacenter == self.local_dc, shuffled) if hosts else ()
+
+
+class FastTokenAwarePolicy(DCAwareRoundRobinPolicy):
+    """
+    Send to any replicas attached to the query, or else fall back to DCAwareRoundRobinPolicy
+    """
+
+    def __init__(self, local_dc='', used_hosts_per_remote_dc=0):
+        DCAwareRoundRobinPolicy.__init__(self, local_dc, used_hosts_per_remote_dc)
+
+    def make_query_plan(self, working_keyspace=None, query=None):
+        """
+        Extend TokenAwarePolicy.make_query_plan() so that we choose the same replicas in preference
+        and most importantly we avoid repeating the (slow) bisect
+        """
+        replicas = query.replicas if hasattr(query, 'replicas') else []
+        for r in replicas:
+            yield r
+
+        for r in DCAwareRoundRobinPolicy.make_query_plan(self, working_keyspace, query):
+            if r not in replicas:
+                yield r
+
 
 class ImportProcess(ChildProcess):
 
@@ -1650,7 +1958,12 @@ class ImportProcess(ChildProcess):
         self.max_attempts = options.copy['maxattempts']
         self.min_batch_size = options.copy['minbatchsize']
         self.max_batch_size = options.copy['maxbatchsize']
+        self.use_prepared_statements = options.copy['preparedstatements']
+        self.dialect_options = options.dialect
         self._session = None
+        self.query = None
+        self.conv = None
+        self.make_statement = None
 
     @property
     def session(self):
@@ -1661,12 +1974,13 @@ class ImportProcess(ChildProcess):
                 cql_version=self.cql_version,
                 protocol_version=self.protocol_version,
                 auth_provider=self.auth_provider,
-                load_balancing_policy=TokenAwarePolicy(DCAwareRoundRobinPolicy(local_dc=self.local_dc)),
+                load_balancing_policy=FastTokenAwarePolicy(local_dc=self.local_dc),
                 ssl_options=ssl_settings(self.hostname, self.config_file) if self.ssl else None,
                 default_retry_policy=ExpBackoffRetryPolicy(self),
                 compression=None,
                 control_connection_timeout=self.connect_timeout,
-                connect_timeout=self.connect_timeout)
+                connect_timeout=self.connect_timeout,
+                idle_heartbeat_interval=0)
 
             self._session = cluster.connect(self.ks)
             self._session.default_timeout = None
@@ -1674,13 +1988,12 @@ class ImportProcess(ChildProcess):
 
     def run(self):
         try:
-            table_meta = self.session.cluster.metadata.keyspaces[self.ks].tables[self.table]
-            is_counter = ("counter" in [table_meta.columns[name].typestring for name in self.valid_columns])
+            pr = profile_on() if PROFILE_ON else None
 
-            if is_counter:
-                self.run_counter(table_meta)
-            else:
-                self.run_normal(table_meta)
+            self.inner_run(*self.make_params())
+
+            if pr:
+                profile_off(pr, file_name='worker_profile_%d.txt' % (os.getpid(),))
 
         except Exception, exc:
             if self.debug:
@@ -1694,67 +2007,88 @@ class ImportProcess(ChildProcess):
             self._session.cluster.shutdown()
         ChildProcess.close(self)
 
-    def run_counter(self, table_meta):
-        """
-        Main run method for tables that contain counter columns.
-        """
-        query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
-
-        # We prepare a query statement to find out the types of the partition key columns so we can
-        # route the update query to the correct replicas. As far as I understood this is the easiest
-        # way to find out the types of the partition columns, we will never use this prepared statement
-        where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
-        select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(self.ks), protect_name(self.table), where_clause)
-        conv = ImportConversion(self, table_meta, self.session.prepare(select_query))
-
-        while True:
-            batch = self.inmsg.get()
-            try:
-                for b in self.split_batches(batch, conv):
-                    self.send_counter_batch(query, conv, b)
+    def make_params(self):
+        metadata = self.session.cluster.metadata
+        table_meta = metadata.keyspaces[self.ks].tables[self.table]
+
+        prepared_statement = None
+        is_counter = ("counter" in [table_meta.columns[name].typestring for name in self.valid_columns])
+        if is_counter:
+            query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
+            make_statement = self.wrap_make_statement(self.make_counter_batch_statement)
+        elif self.use_prepared_statements:
+            query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
+                                                            protect_name(self.table),
+                                                            ', '.join(protect_names(self.valid_columns),),
+                                                            ', '.join(['?' for _ in self.valid_columns]))
+
+            query = self.session.prepare(query)
+            query.consistency_level = self.consistency_level
+            prepared_statement = query
+            make_statement = self.wrap_make_statement(self.make_prepared_batch_statement)
+        else:
+            query = 'INSERT INTO %s.%s (%s) VALUES (%%s)' % (protect_name(self.ks),
+                                                             protect_name(self.table),
+                                                             ', '.join(protect_names(self.valid_columns),))
+            make_statement = self.wrap_make_statement(self.make_non_prepared_batch_statement)
 
-            except Exception, exc:
-                self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                if self.debug:
-                    traceback.print_exc(exc)
+        conv = ImportConversion(self, table_meta, prepared_statement)
+        tm = TokenMap(self.ks, self.hostname, self.local_dc, self.session)
+        return query, conv, tm, make_statement
 
-    def run_normal(self, table_meta):
+    def inner_run(self, query, conv, tm, make_statement):
         """
-        Main run method for normal tables, i.e. tables that do not contain counter columns.
+        Main run method. Note that we bind self methods that are called inside loops
+        for performance reasons.
         """
-        query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
-                                                        protect_name(self.table),
-                                                        ', '.join(protect_names(self.valid_columns),),
-                                                        ', '.join(['?' for _ in self.valid_columns]))
+        self.query = query
+        self.conv = conv
+        self.make_statement = make_statement
 
-        query_statement = self.session.prepare(query)
-        query_statement.consistency_level = self.consistency_level
-        conv = ImportConversion(self, table_meta, query_statement)
+        convert_rows = self.convert_rows
+        split_into_batches = self.split_into_batches
+        result_callback = self.result_callback
+        err_callback = self.err_callback
+        session = self.session
 
         while True:
-            batch = self.inmsg.get()
+            chunk = self.inmsg.recv()
+            if chunk is None:
+                break
+
             try:
-                for b in self.split_batches(batch, conv):
-                    self.send_normal_batch(conv, query_statement, b)
+                chunk['rows'] = convert_rows(conv, chunk)
+                for replicas, batch in split_into_batches(chunk, conv, tm):
+                    statement = make_statement(query, conv, chunk, batch, replicas)
+                    future = session.execute_async(statement)
+                    future.add_callbacks(callback=result_callback, callback_args=(batch, chunk),
+                                         errback=err_callback, errback_args=(batch, chunk, replicas))
 
             except Exception, exc:
-                self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                if self.debug:
-                    traceback.print_exc(exc)
+                self.report_error(exc, chunk, chunk['rows'])
 
-    def send_counter_batch(self, query_text, conv, batch):
-        if self.test_failures and self.maybe_inject_failures(batch):
-            return
+    def wrap_make_statement(self, inner_make_statement):
+        def make_statement(query, conv, chunk, batch, replicas):
+            try:
+                return inner_make_statement(query, conv, batch, replicas)
+            except Exception, exc:
+                print "Failed to make batch statement: {}".format(exc)
+                self.report_error(exc, chunk, batch['rows'])
+                return None
 
-        error_rows = []
-        batch_statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+        def make_statement_with_failures(query, conv, chunk, batch, replicas):
+            failed_batch = self.maybe_inject_failures(batch)
+            if failed_batch:
+                return failed_batch
+            return make_statement(query, conv, chunk, batch, replicas)
 
-        for r in batch['rows']:
-            row = self.filter_row_values(r)
-            if len(row) != len(self.valid_columns):
-                error_rows.append(row)
-                continue
+        return make_statement_with_failures if self.test_failures else make_statement
 
+    def make_counter_batch_statement(self, query, conv, batch, replicas):
+        statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+        statement.replicas = replicas
+        statement.keyspace = self.ks
+        for row in batch['rows']:
             where_clause = []
             set_clause = []
             for i, value in enumerate(row):
@@ -1763,65 +2097,61 @@ class ImportProcess(ChildProcess):
                 else:
                     set_clause.append("%s=%s+%s" % (self.valid_columns[i], self.valid_columns[i], value))
 
-            full_query_text = query_text % (','.join(set_clause), ' AND '.join(where_clause))
-            batch_statement.add(full_query_text)
-
-        self.execute_statement(batch_statement, batch)
+            full_query_text = query % (','.join(set_clause), ' AND '.join(where_clause))
+            statement.add(full_query_text)
+        return statement
 
-        if error_rows:
-            self.outmsg.put((ImportTask.split_batch(batch, error_rows),
-                            '%s - %s' % (ParseError.__name__, "Failed to parse one or more rows")))
+    def make_prepared_batch_statement(self, query, _, batch, replicas):
+        """
+        Return a batch statement. This is an optimized version of:
 
-    def send_normal_batch(self, conv, query_statement, batch):
-        if self.test_failures and self.maybe_inject_failures(batch):
-            return
+            statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+            for row in batch['rows']:
+                statement.add(query, row)
 
-        good_rows, converted_rows, errors = self.convert_rows(conv, batch['rows'])
+        We could optimize further by removing bound_statements altogether but we'd have to duplicate much
+        more driver's code (BoundStatement.bind()).
+        """
+        statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+        statement.replicas = replicas
+        statement.keyspace = self.ks
+        statement._statements_and_parameters = [(True, query.query_id, query.bind(r).values) for r in batch['rows']]
+        return statement
 
-        if converted_rows:
-            try:
-                statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
-                for row in converted_rows:
-                    statement.add(query_statement, row)
-                self.execute_statement(statement, ImportTask.split_batch(batch, good_rows))
-            except Exception, exc:
-                self.err_callback(exc, ImportTask.split_batch(batch, good_rows))
+    def make_non_prepared_batch_statement(self, query, _, batch, replicas):
+        statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+        statement.replicas = replicas
+        statement.keyspace = self.ks
+        statement._statements_and_parameters = [(False, query % (','.join(r),), ()) for r in batch['rows']]
+        return statement
 
-        if errors:
-            for msg, rows in errors.iteritems():
-                self.outmsg.put((ImportTask.split_batch(batch, rows),
-                                '%s - %s' % (ParseError.__name__, msg)))
-
-    def convert_rows(self, conv, rows):
+    def convert_rows(self, conv, chunk):
         """
-        Try to convert each row. If conversion is OK then add the converted result to converted_rows
-        and the original string to good_rows. Else add the original string to error_rows. Return the three
-        arrays.
+        Return converted rows and report any errors during conversion.
         """
-        good_rows = []
-        errors = defaultdict(list)
-        converted_rows = []
+        def filter_row_values(row):
+            return [v for i, v in enumerate(row) if i not in self.skip_column_indexes]
 
-        for r in rows:
-            row = self.filter_row_values(r)
-            if len(row) != len(self.valid_columns):
-                msg = 'Invalid row length %d should be %d' % (len(row), len(self.valid_columns))
-                errors[msg].append(row)
-                continue
+        if self.skip_column_indexes:
+            rows = [filter_row_values(r) for r in list(csv.reader(chunk['rows'], **self.dialect_options))]
+        else:
+            rows = list(csv.reader(chunk['rows'], **self.dialect_options))
 
-            try:
-                converted_rows.append(conv.get_row_values(row))
-                good_rows.append(row)
-            except ParseError, err:
-                errors[err.message].append(row)
+        errors = defaultdict(list)
 
-        return good_rows, converted_rows, errors
+        def convert_row(r):
+            try:
+                return conv.convert_row(r)
+            except Exception, err:
+                errors[err.message].append(r)
+                return None
 
-    def filter_row_values(self, row):
-        if not self.skip_column_indexes:
-            return row
+        converted_rows = filter(None, [convert_row(r) for r in rows])
 
-        return [v for i, v in enumerate(row) if i not in self.skip_column_indexes]
+        if errors:
+            for msg, rows in errors.iteritems():
+                self.report_error(ParseError(msg), chunk, rows)
+        return converted_rows
 
     def maybe_inject_failures(self, batch):
         """
@@ -1836,86 +2166,94 @@ class ImportProcess(ChildProcess):
                 if batch['attempts'] < failing_batch['failures']:
                     statement = SimpleStatement("INSERT INTO badtable (a, b) VALUES (1, 2)",
                                                 consistency_level=self.consistency_level)
-                    self.execute_statement(statement, batch)
-                    return True
+                    return statement
 
         if 'exit_batch' in self.test_failures:
             exit_batch = self.test_failures['exit_batch']
             if exit_batch['id'] == batch['id']:
                 sys.exit(1)
 
-        return False  # carry on as normal
+        return None  # carry on as normal
 
-    def execute_statement(self, statement, batch):
-        future = self.session.execute_async(statement)
-        future.add_callbacks(callback=self.result_callback, callback_args=(batch, ),
-                             errback=self.err_callback, errback_args=(batch, ))
+    @staticmethod
+    def make_batch(batch_id, rows, attempts=1):
+        return {'id': batch_id, 'rows': rows, 'attempts': attempts}
 
-    def split_batches(self, batch, conv):
+    def split_into_batches(self, chunk, conv, tm):
         """
-        Batch rows by partition key, if there are at least min_batch_size (2)
-        rows with the same partition key. These batches can be as big as they want
-        since this translates to a single insert operation server side.
-
-        If there are less than min_batch_size rows for a partition, work out the
-        first replica for this partition and add the rows to replica left-over rows.
-
-        Then batch the left-overs of each replica up to max_batch_size.
+        Batch rows by ring position or replica.
+        If there are at least min_batch_size rows for a ring position then split these rows into
+        groups of max_batch_size and send a batch for each group, using all replicas for this ring position.
+        Otherwise, we are forced to batch by replica, and here unfortunately we can only choose one replica to
+        guarantee common replicas across partition keys. We are typically able
+        to batch by ring position for small clusters or when VNODES are not used. For large clusters with VNODES
+        it may not be possible, in this case it helps to increase the CHUNK SIZE but up to a limit, otherwise
+        we may choke the cluster.
         """
-        rows_by_pk = defaultdict(list)
+
+        rows_by_ring_pos = defaultdict(list)
         errors = defaultdict(list)
 
-        for row in batch['rows']:
+        min_batch_size = self.min_batch_size
+        max_batch_size = self.max_batch_size
+        ring = tm.ring
+
+        get_row_partition_key_values = conv.get_row_partition_key_values_fcn()
+        pk_to_token_value = tm.pk_to_token_value
+        get_ring_pos = tm.get_ring_pos
+        make_batch = self.make_batch
+
+        for row in chunk['rows']:
             try:
-                pk = conv.get_row_partition_key_values(row)
-                rows_by_pk[pk].append(row)
-            except ParseError, e:
+                pk = get_row_partition_key_values(row)
+                rows_by_ring_pos[get_ring_pos(ring, pk_to_token_value(pk))].append(row)
+            except Exception, e:
                 errors[e.message].append(row)
 
         if errors:
             for msg, rows in errors.iteritems():
-                self.outmsg.put((ImportTask.split_batch(batch, rows),
-                                 '%s - %s' % (ParseError.__name__, msg)))
+                self.report_error(ParseError(msg), chunk, rows)
 
+        replicas = tm.replicas
+        filter_replicas = tm.filter_replicas
         rows_by_replica = defaultdict(list)
-        for pk, rows in rows_by_pk.iteritems():
-            if len(rows) >= self.min_batch_size:
-                yield ImportTask.make_batch(batch['id'], rows, batch['attempts'])
+        for ring_pos, rows in rows_by_ring_pos.iteritems():
+            if len(rows) > min_batch_size:
+                for i in xrange(0, len(rows), max_batch_size):
+                    yield filter_replicas(replicas[ring_pos]), make_batch(chunk['id'], rows[i:i + max_batch_size])
             else:
-                replica = self.get_replica(pk)
-                rows_by_replica[replica].extend(rows)
-
-        for replica, rows in rows_by_replica.iteritems():
-            for b in self.batches(rows, batch):
-                yield b
-
-    def get_replica(self, pk):
-        """
-        Return the first replica or the host we are already connected to if there are no local
-        replicas that are up. We always use the first replica to match the replica chosen by the driver
-        TAR, see TokenAwarePolicy.make_query_plan().
-        """
-        metadata = self.session.cluster.metadata
-        replicas = filter(lambda r: r.is_up and r.datacenter == self.local_dc, metadata.get_replicas(self.ks, pk))
-        ret = replicas[0].address if len(replicas) > 0 else self.hostname
-        return ret
-
-    def batches(self, rows, batch):
-        """
-        Split rows into batches of max_batch_size
-        """
-        for i in xrange(0, len(rows), self.max_batch_size):
-            yield ImportTask.make_batch(batch['id'], rows[i:i + self.max_batch_size], batch['attempts'])
-
-    def result_callback(self, _, batch):
-        batch['imported'] = len(batch['rows'])
-        batch['rows'] = []  # no need to resend these, just send the count in 'imported'
-        self.outmsg.put((batch, None))
-
-    def err_callback(self, response, batch):
-        self.outmsg.put((batch, '%s - %s' % (response.__class__.__name__, response.message)))
+                # select only the first valid replica to guarantee more overlap or none at all
+                rows_by_replica[filter_replicas(replicas[ring_pos])[:1]].extend(rows)
+
+        # Now send the batches by replica
+        for replicas, rows in rows_by_replica.iteritems():
+            for i in xrange(0, len(rows), max_batch_size):
+                yield replicas, make_batch(chunk['id'], rows[i:i + max_batch_size])
+
+    def result_callback(self, _, batch, chunk):
+        self.update_chunk(batch['rows'], chunk)
+
+    def err_callback(self, response, batch, chunk, replicas):
+        err_is_final = batch['attempts'] >= self.max_attempts
+        self.report_error(response, chunk, batch['rows'], batch['attempts'], err_is_final)
+        if not err_is_final:
+            batch['attempts'] += 1
+            statement = self.make_statement(self.query, self.conv, chunk, batch, replicas)
+            future = self.session.execute_async(statement)
+            future.add_callbacks(callback=self.result_callback, callback_args=(batch, chunk),
+                                 errback=self.err_callback, errback_args=(batch, chunk, replicas))
+
+    def report_error(self, err, chunk, rows=None, attempts=1, final=True):
         if self.debug:
-            traceback.print_exc(response)
+            traceback.print_exc(err)
+        self.outmsg.send(ImportTaskError(err.__class__.__name__, err.message, rows, attempts, final))
+        if final:
+            self.update_chunk(rows, chunk)
+
+    def update_chunk(self, rows, chunk):
+        chunk['imported'] += len(rows)
+        if chunk['imported'] == chunk['num_rows_sent']:
+            self.outmsg.send(ImportProcessResult(chunk['num_rows_sent']))
 
 
 class RateMeter(object):
@@ -1937,11 +2275,19 @@ class RateMeter(object):
         self.current_record += n
         self.maybe_update()
 
-    def maybe_update(self):
+    def maybe_update(self, sleep=False):
+        if self.current_record == 0:
+            return
+
         new_checkpoint_time = time.time()
-        if new_checkpoint_time - self.last_checkpoint_time >= self.update_interval:
+        time_difference = new_checkpoint_time - self.last_checkpoint_time
+        if time_difference >= self.update_interval:
             self.update(new_checkpoint_time)
             self.log_message()
+        elif sleep:
+            remaining_time = time_difference - self.update_interval
+            if remaining_time > 0.000001:
+                time.sleep(remaining_time)
 
     def update(self, new_checkpoint_time):
         time_difference = new_checkpoint_time - self.last_checkpoint_time

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/pylib/cqlshlib/util.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/util.py b/pylib/cqlshlib/util.py
index 281aad6..3ee128d 100644
--- a/pylib/cqlshlib/util.py
+++ b/pylib/cqlshlib/util.py
@@ -23,6 +23,12 @@ from itertools import izip
 from datetime import timedelta, tzinfo
 from StringIO import StringIO
 
+try:
+    from line_profiler import LineProfiler
+    HAS_LINE_PROFILER = True
+except ImportError:
+    HAS_LINE_PROFILER = False
+
 ZERO = timedelta(0)
 
 
@@ -126,18 +132,35 @@ def get_file_encoding_bomsize(filename):
     else:
         file_encoding, size = "utf-8", 0
 
-    return (file_encoding, size)
+    return file_encoding, size
+
 
+def profile_on(fcn_names=None):
+    if fcn_names and HAS_LINE_PROFILER:
+        pr = LineProfiler()
+        for fcn_name in fcn_names:
+            pr.add_function(fcn_name)
+        pr.enable()
+        return pr
 
-def profile_on():
     pr = cProfile.Profile()
     pr.enable()
     return pr
 
 
-def profile_off(pr):
+def profile_off(pr, file_name):
     pr.disable()
     s = StringIO()
-    ps = pstats.Stats(pr, stream=s).sort_stats('cumulative')
-    ps.print_stats()
-    print s.getvalue()
+
+    if HAS_LINE_PROFILER and isinstance(pr, LineProfiler):
+        pr.print_stats(s)
+    else:
+        ps = pstats.Stats(pr, stream=s).sort_stats('cumulative')
+        ps.print_stats()
+
+    ret = s.getvalue()
+    if file_name:
+        with open(file_name, 'w') as f:
+            print "Writing to %s\n" % (f.name, )
+            f.write(ret)
+    return ret

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/pylib/setup.py
----------------------------------------------------------------------
diff --git a/pylib/setup.py b/pylib/setup.py
index 704d077..3654502 100755
--- a/pylib/setup.py
+++ b/pylib/setup.py
@@ -16,9 +16,11 @@
 # limitations under the License.
 
 from distutils.core import setup
+from Cython.Build import cythonize
 
 setup(
     name="cassandra-pylib",
     description="Cassandra Python Libraries",
     packages=["cqlshlib"],
+    ext_modules=cythonize("cqlshlib/copyutil.py"),
 )


[23/23] cassandra git commit: Merge branch 'cassandra-3.5' into trunk

Posted by sl...@apache.org.
Merge branch 'cassandra-3.5' into trunk


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/acde5081
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/acde5081
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/acde5081

Branch: refs/heads/trunk
Commit: acde5081595d1e47403563dadd21ddbca5ef5e8d
Parents: 733d1ee 6329d54
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue Mar 8 10:26:08 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Mar 8 10:26:08 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                |    1 +
 bin/cqlsh.py               |   33 +-
 pylib/cqlshlib/copyutil.py | 1173 +++++++++++++++++++++++++--------------
 pylib/cqlshlib/util.py     |   35 +-
 pylib/setup.py             |    2 +
 5 files changed, 824 insertions(+), 420 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/acde5081/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/acde5081/bin/cqlsh.py
----------------------------------------------------------------------


[05/23] cassandra git commit: COPY FROM on large datasets: fix progress report and debug performance

Posted by sl...@apache.org.
COPY FROM on large datasets: fix progress report and debug performance

patch by Stefania Alborghetti; reviewed by Adam Holmberg for CASSANDRA-11053


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c3d2f26f
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c3d2f26f
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c3d2f26f

Branch: refs/heads/cassandra-3.5
Commit: c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc
Parents: 0129f70
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Thu Jan 28 14:31:55 2016 +0800
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Mar 8 10:19:13 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                |    1 +
 bin/cqlsh                  |   28 +-
 pylib/cqlshlib/copyutil.py | 1160 +++++++++++++++++++++++++--------------
 pylib/cqlshlib/util.py     |   35 +-
 pylib/setup.py             |    2 +
 5 files changed, 809 insertions(+), 417 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index eed9035..d6b085c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.14
+ * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
  * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
  * Don't remove FailureDetector history on removeEndpoint (CASSANDRA-10371)
  * Only notify if repair status changed (CASSANDRA-11172)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/bin/cqlsh
----------------------------------------------------------------------
diff --git a/bin/cqlsh b/bin/cqlsh
index 7a39636..374e588 100755
--- a/bin/cqlsh
+++ b/bin/cqlsh
@@ -417,7 +417,7 @@ COPY_COMMON_OPTIONS = ['DELIMITER', 'QUOTE', 'ESCAPE', 'HEADER', 'NULL', 'DATETI
                        'MAXATTEMPTS', 'REPORTFREQUENCY', 'DECIMALSEP', 'THOUSANDSSEP', 'BOOLSTYLE',
                        'NUMPROCESSES', 'CONFIGFILE', 'RATEFILE']
 COPY_FROM_OPTIONS = ['CHUNKSIZE', 'INGESTRATE', 'MAXBATCHSIZE', 'MINBATCHSIZE', 'MAXROWS',
-                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE']
+                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE', 'PREPAREDSTATEMENTS']
 COPY_TO_OPTIONS = ['ENCODING', 'PAGESIZE', 'PAGETIMEOUT', 'BEGINTOKEN', 'ENDTOKEN', 'MAXOUTPUTSIZE', 'MAXREQUESTS']
 
 
@@ -533,8 +533,23 @@ def insert_driver_hooks():
 
 def extend_cql_deserialization():
     """
-    The python driver returns BLOBs as string, but we expect them as bytearrays
+    The python driver returns BLOBs as string, but we expect them as bytearrays; therefore we change
+    the implementation of cassandra.cqltypes.BytesType.deserialize.
+
+    The deserializers package exists only when the driver has been compiled with cython extensions and
+    cassandra.deserializers.DesBytesType replaces cassandra.cqltypes.BytesType.deserialize.
+
+    DesBytesTypeByteArray is a fast deserializer that converts blobs into bytearrays but it was
+    only introduced recently (3.1.0). If it is available we use it, otherwise we remove
+    cassandra.deserializers.DesBytesType so that we fall back onto cassandra.cqltypes.BytesType.deserialize
+    just like in the case where no cython extensions are present.
     """
+    if hasattr(cassandra, 'deserializers'):
+        if hasattr(cassandra.deserializers, 'DesBytesTypeByteArray'):
+            cassandra.deserializers.DesBytesType = cassandra.deserializers.DesBytesTypeByteArray
+        else:
+            del cassandra.deserializers.DesBytesType
+
     cassandra.cqltypes.BytesType.deserialize = staticmethod(lambda byts, protocol_version: bytearray(byts))
     cassandra.cqltypes.CassandraType.support_empty_values = True
 
@@ -1534,9 +1549,9 @@ class Shell(cmd.Cmd):
 
         Available COPY FROM options and defaults:
 
-          CHUNKSIZE=1000          - the size of chunks passed to worker processes
+          CHUNKSIZE=5000          - the size of chunks passed to worker processes
           INGESTRATE=100000       - an approximate ingest rate in rows per second
-          MINBATCHSIZE=2          - the minimum size of an import batch
+          MINBATCHSIZE=10         - the minimum size of an import batch
           MAXBATCHSIZE=20         - the maximum size of an import batch
           MAXROWS=-1              - the maximum number of rows, -1 means no maximum
           SKIPROWS=0              - the number of rows to skip
@@ -1545,6 +1560,11 @@ class Shell(cmd.Cmd):
           MAXINSERTERRORS=-1      - the maximum global number of insert errors, -1 means no maximum
           ERRFILE=''              - a file where to store all rows that could not be imported, by default this is
                                     import_ks_table.err where <ks> is your keyspace and <table> is your table name.
+          PREPAREDSTATEMENTS=True - whether to use prepared statements when importing, by default True. Set this to
+                                    False if you don't mind shifting data parsing to the cluster. The cluster will also
+                                    have to compile every batch statement. For large and oversized clusters
+                                    this will result in a faster import but for smaller clusters it may generate
+                                    timeouts.
 
         Available COPY TO options and defaults:
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/pylib/cqlshlib/copyutil.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/copyutil.py b/pylib/cqlshlib/copyutil.py
index f9e4a85..cd03765 100644
--- a/pylib/cqlshlib/copyutil.py
+++ b/pylib/cqlshlib/copyutil.py
@@ -1,3 +1,5 @@
+# cython: profile=True
+
 # Licensed to the Apache Software Foundation (ASF) under one
 # or more contributor license agreements.  See the NOTICE file
 # distributed with this work for additional information
@@ -21,25 +23,29 @@ import json
 import glob
 import multiprocessing as mp
 import os
-import Queue
+import platform
+import random
 import re
 import struct
 import sys
 import time
 import traceback
 
+from bisect import bisect_right
 from calendar import timegm
-from collections import defaultdict, deque, namedtuple
+from collections import defaultdict, namedtuple
 from decimal import Decimal
 from random import randrange
 from StringIO import StringIO
+from select import select
 from threading import Lock
 from uuid import UUID
+from util import profile_on, profile_off
 
 from cassandra.cluster import Cluster
 from cassandra.cqltypes import ReversedType, UserType
-from cassandra.metadata import protect_name, protect_names
-from cassandra.policies import RetryPolicy, WhiteListRoundRobinPolicy, TokenAwarePolicy, DCAwareRoundRobinPolicy
+from cassandra.metadata import protect_name, protect_names, protect_value
+from cassandra.policies import RetryPolicy, WhiteListRoundRobinPolicy, DCAwareRoundRobinPolicy
 from cassandra.query import BatchStatement, BatchType, SimpleStatement, tuple_factory
 from cassandra.util import Date, Time
 
@@ -48,6 +54,10 @@ from displaying import NO_COLOR_MAP
 from formatting import format_value_default, EMPTY, get_formatter
 from sslhandling import ssl_settings
 
+PROFILE_ON = False
+STRACE_ON = False
+IS_LINUX = platform.system() == 'Linux'
+
 CopyOptions = namedtuple('CopyOptions', 'copy dialect unrecognized')
 
 
@@ -59,6 +69,81 @@ def safe_normpath(fname):
     return os.path.normpath(os.path.expanduser(fname)) if fname else fname
 
 
+class OneWayChannel(object):
+    """
+    A one way pipe protected by two process level locks, one for reading and one for writing.
+    """
+    def __init__(self):
+        self.reader, self.writer = mp.Pipe(duplex=False)
+        self.rlock = mp.Lock()
+        self.wlock = mp.Lock()
+
+    def send(self, obj):
+        with self.wlock:
+            self.writer.send(obj)
+
+    def recv(self):
+        with self.rlock:
+            return self.reader.recv()
+
+    def close(self):
+        self.reader.close()
+        self.writer.close()
+
+
+class OneWayChannels(object):
+    """
+    A group of one way channels.
+    """
+    def __init__(self, num_channels):
+        self.channels = [OneWayChannel() for _ in xrange(num_channels)]
+        self._readers = [ch.reader for ch in self.channels]
+        self._rlocks = [ch.rlock for ch in self.channels]
+        self._rlocks_by_readers = dict([(ch.reader, ch.rlock) for ch in self.channels])
+        self.num_channels = num_channels
+
+        self.recv = self.recv_select if IS_LINUX else self.recv_polling
+
+    def recv_select(self, timeout):
+        """
+        Implementation of the recv method for Linux, where select is available. Receive an object from
+        all pipes that are ready for reading without blocking.
+        """
+        readable, _, _ = select(self._readers, [], [], timeout)
+        for r in readable:
+            with self._rlocks_by_readers[r]:
+                try:
+                    yield r.recv()
+                except EOFError:
+                    continue
+
+    def recv_polling(self, timeout):
+        """
+        Implementation of the recv method for platforms where select() is not available for pipes.
+        We poll on all of the readers with a very small timeout. We stop when the timeout specified
+        has been received but we may exceed it since we check all processes during each sweep.
+        """
+        start = time.time()
+        while True:
+            for i, r in enumerate(self._readers):
+                with self._rlocks[i]:
+                    if r.poll(0.000000001):
+                        try:
+                            yield r.recv()
+                        except EOFError:
+                            continue
+
+            if time.time() - start > timeout:
+                break
+
+    def close(self):
+        for ch in self.channels:
+            try:
+                ch.close()
+            except:
+                pass
+
+
 class CopyTask(object):
     """
     A base class for ImportTask and ExportTask
@@ -72,15 +157,18 @@ class CopyTask(object):
         self.protocol_version = protocol_version
         self.config_file = config_file
         # do not display messages when exporting to STDOUT
-        self.printmsg = self._printmsg if self.fname is not None or direction == 'in' else lambda _, eol='\n': None
+        self.printmsg = self._printmsg if self.fname is not None or direction == 'from' else lambda _, eol='\n': None
         self.options = self.parse_options(opts, direction)
 
         self.num_processes = self.options.copy['numprocesses']
+        if direction == 'in':
+            self.num_processes += 1  # add the feeder process
+
         self.printmsg('Using %d child processes' % (self.num_processes,))
 
         self.processes = []
-        self.inmsg = mp.Queue()
-        self.outmsg = mp.Queue()
+        self.inmsg = OneWayChannels(self.num_processes)
+        self.outmsg = OneWayChannels(self.num_processes)
 
         self.columns = CopyTask.get_columns(shell, ks, table, columns)
         self.time_start = time.time()
@@ -166,10 +254,10 @@ class CopyTask(object):
         copy_options['maxattempts'] = int(opts.pop('maxattempts', 5))
         copy_options['dtformats'] = opts.pop('datetimeformat', shell.display_time_format)
         copy_options['float_precision'] = shell.display_float_precision
-        copy_options['chunksize'] = int(opts.pop('chunksize', 1000))
+        copy_options['chunksize'] = int(opts.pop('chunksize', 5000))
         copy_options['ingestrate'] = int(opts.pop('ingestrate', 100000))
         copy_options['maxbatchsize'] = int(opts.pop('maxbatchsize', 20))
-        copy_options['minbatchsize'] = int(opts.pop('minbatchsize', 2))
+        copy_options['minbatchsize'] = int(opts.pop('minbatchsize', 10))
         copy_options['reportfrequency'] = float(opts.pop('reportfrequency', 0.25))
         copy_options['consistencylevel'] = shell.consistency_level
         copy_options['decimalsep'] = opts.pop('decimalsep', '.')
@@ -186,6 +274,7 @@ class CopyTask(object):
         copy_options['errfile'] = safe_normpath(opts.pop('errfile', 'import_%s_%s.err' % (self.ks, self.table,)))
         copy_options['ratefile'] = safe_normpath(opts.pop('ratefile', ''))
         copy_options['maxoutputsize'] = int(opts.pop('maxoutputsize', '-1'))
+        copy_options['preparedstatements'] = bool(opts.pop('preparedstatements', 'true').lower() == 'true')
 
         self.check_options(copy_options)
         return CopyOptions(copy=copy_options, dialect=dialect_options, unrecognized=opts)
@@ -206,14 +295,17 @@ class CopyTask(object):
     def get_num_processes(cap):
         """
         Pick a reasonable number of child processes. We need to leave at
-        least one core for the parent process.  This doesn't necessarily
-        need to be capped, but 4 is currently enough to keep
-        a single local Cassandra node busy so we use this for import, whilst
-        for export we use 16 since we can connect to multiple Cassandra nodes.
-        Eventually this parameter will become an option.
+        least one core for the parent process.
+        """
+        return max(1, min(cap, CopyTask.get_num_cores() - 1))
+
+    @staticmethod
+    def get_num_cores():
+        """
+        Return the number of cores if available.
         """
         try:
-            return max(1, min(cap, mp.cpu_count() - 1))
+            return mp.cpu_count()
         except NotImplementedError:
             return 1
 
@@ -244,28 +336,40 @@ class CopyTask(object):
         return shell.get_column_names(ks, table) if not columns else columns
 
     def close(self):
-        for process in self.processes:
-            process.terminate()
-
+        self.stop_processes()
         self.inmsg.close()
         self.outmsg.close()
 
     def num_live_processes(self):
         return sum(1 for p in self.processes if p.is_alive())
 
+    @staticmethod
+    def get_pid():
+        return os.getpid() if hasattr(os, 'getpid') else None
+
+    @staticmethod
+    def trace_process(pid):
+        if pid and STRACE_ON:
+            os.system("strace -vvvv -c -o strace.{pid}.out -e trace=all -p {pid}&".format(pid=pid))
+
+    def start_processes(self):
+        for i, process in enumerate(self.processes):
+            process.start()
+            self.trace_process(process.pid)
+
+        self.trace_process(self.get_pid())
+
+    def stop_processes(self):
+        for process in self.processes:
+            process.terminate()
+
     def make_params(self):
         """
         Return a dictionary of parameters to be used by the worker processes.
         On Windows this dictionary must be pickle-able.
-
-        inmsg is the message queue flowing from parent to child process, so outmsg from the parent point
-        of view and, vice-versa,  outmsg is the message queue flowing from child to parent, so inmsg
-        from the parent point of view, hence the two are swapped below.
         """
         shell = self.shell
-        return dict(inmsg=self.outmsg,  # see comment above
-                    outmsg=self.inmsg,  # see comment above
-                    ks=self.ks,
+        return dict(ks=self.ks,
                     table=self.table,
                     local_dc=self.local_dc,
                     columns=self.columns,
@@ -281,6 +385,17 @@ class CopyTask(object):
                     debug=shell.debug
                     )
 
+    def update_params(self, params, i):
+        """
+        Add the communication channels to the parameters to be passed to the worker process:
+            inmsg is the message queue flowing from parent to child process, so outmsg from the parent point
+            of view and, vice-versa,  outmsg is the message queue flowing from child to parent, so inmsg
+            from the parent point of view, hence the two are swapped below.
+        """
+        params['inmsg'] = self.outmsg.channels[i]
+        params['outmsg'] = self.inmsg.channels[i]
+        return params
+
 
 class ExportWriter(object):
     """
@@ -414,10 +529,9 @@ class ExportTask(CopyTask):
 
         params = self.make_params()
         for i in xrange(self.num_processes):
-            self.processes.append(ExportProcess(params))
+            self.processes.append(ExportProcess(self.update_params(params, i)))
 
-        for process in self.processes:
-            process.start()
+        self.start_processes()
 
         try:
             self.export_records(ranges)
@@ -468,11 +582,12 @@ class ExportTask(CopyTask):
 
             return ret
 
-        def make_range_data(replicas=[]):
+        def make_range_data(replicas=None):
             hosts = []
-            for r in replicas:
-                if r.is_up and r.datacenter == local_dc:
-                    hosts.append(r.address)
+            if replicas:
+                for r in replicas:
+                    if r.is_up and r.datacenter == local_dc:
+                        hosts.append(r.address)
             if not hosts:
                 hosts.append(hostname)  # fallback to default host if no replicas in current dc
             return {'hosts': tuple(hosts), 'attempts': 0, 'rows': 0}
@@ -542,10 +657,13 @@ class ExportTask(CopyTask):
             return None
 
     def send_work(self, ranges, tokens_to_send):
+        i = 0
         for token_range in tokens_to_send:
-            self.outmsg.put((token_range, ranges[token_range]))
+            self.outmsg.channels[i].send((token_range, ranges[token_range]))
             ranges[token_range]['attempts'] += 1
 
+            i = i + 1 if i < self.num_processes - 1 else 0
+
     def export_records(self, ranges):
         """
         Send records to child processes and monitor them by collecting their results
@@ -568,8 +686,7 @@ class ExportTask(CopyTask):
         succeeded = 0
         failed = 0
         while (failed + succeeded) < total_requests and self.num_live_processes() == num_processes:
-            try:
-                token_range, result = self.inmsg.get(timeout=1.0)
+            for token_range, result in self.inmsg.recv(timeout=0.1):
                 if token_range is None and result is None:  # a request has finished
                     succeeded += 1
                 elif isinstance(result, Exception):  # an error occurred
@@ -594,8 +711,6 @@ class ExportTask(CopyTask):
                     self.writer.write(data, num)
                     meter.increment(n=num)
                     ranges[token_range]['rows'] += num
-            except Queue.Empty:
-                pass
 
         if self.num_live_processes() < len(processes):
             for process in processes:
@@ -612,7 +727,7 @@ class ExportTask(CopyTask):
                        self.describe_interval(time.time() - self.time_start)))
 
 
-class ImportReader(object):
+class FilesReader(object):
     """
     A wrapper around a csv reader to keep track of when we have
     exhausted reading input files. We are passed a comma separated
@@ -620,18 +735,15 @@ class ImportReader(object):
     We generate a source generator and we read each source one
     by one.
     """
-    def __init__(self, task):
-        self.shell = task.shell
-        self.options = task.options
-        self.printmsg = task.printmsg
-        self.chunk_size = self.options.copy['chunksize']
-        self.header = self.options.copy['header']
-        self.max_rows = self.options.copy['maxrows']
-        self.skip_rows = self.options.copy['skiprows']
-        self.sources = self.get_source(task.fname)
+    def __init__(self, fname, options):
+        self.chunk_size = options.copy['chunksize']
+        self.header = options.copy['header']
+        self.max_rows = options.copy['maxrows']
+        self.skip_rows = options.copy['skiprows']
+        self.fname = fname
+        self.sources = None  # must be created later due to pickle problems on Windows
         self.num_sources = 0
         self.current_source = None
-        self.current_reader = None
         self.num_read = 0
 
     def get_source(self, paths):
@@ -640,35 +752,33 @@ class ImportReader(object):
          wrapping the source input, file name and a boolean indicating
          if it requires closing.
         """
-        shell = self.shell
-        LineSource = namedtuple('LineSource', 'input close fname')
-
         def make_source(fname):
             try:
-                ret = LineSource(input=open(fname, 'rb'), close=True, fname=fname)
-                return ret
+                return open(fname, 'rb')
             except IOError, e:
-                shell.printerr("Can't open %r for reading: %s" % (fname, e))
+                self.printmsg("Can't open %r for reading: %s" % (fname, e))
                 return None
 
-        if paths is None:
-            self.printmsg("[Use \. on a line by itself to end input]")
-            yield LineSource(input=shell.use_stdin_reader(prompt='[copy] ', until=r'\.'), close=False, fname='')
-        else:
-            for path in paths.split(','):
-                path = path.strip()
-                if os.path.isfile(path):
-                    yield make_source(path)
-                else:
-                    for f in glob.glob(path):
-                        yield (make_source(f))
+        for path in paths.split(','):
+            path = path.strip()
+            if os.path.isfile(path):
+                yield make_source(path)
+            else:
+                for f in glob.glob(path):
+                    yield (make_source(f))
+
+    @staticmethod
+    def printmsg(msg, eol='\n'):
+        sys.stdout.write(msg + eol)
+        sys.stdout.flush()
 
     def start(self):
+        self.sources = self.get_source(self.fname)
         self.next_source()
 
     @property
     def exhausted(self):
-        return not self.current_reader
+        return not self.current_source
 
     def next_source(self):
         """
@@ -679,40 +789,34 @@ class ImportReader(object):
         while self.current_source is None:
             try:
                 self.current_source = self.sources.next()
-                if self.current_source and self.current_source.fname:
+                if self.current_source:
                     self.num_sources += 1
             except StopIteration:
                 return False
 
         if self.header:
-            self.current_source.input.next()
+            self.current_source.next()
 
-        self.current_reader = csv.reader(self.current_source.input, **self.options.dialect)
         return True
 
     def close_current_source(self):
         if not self.current_source:
             return
 
-        if self.current_source.close:
-            self.current_source.input.close()
-        elif self.shell.tty:
-            print
-
+        self.current_source.close()
         self.current_source = None
-        self.current_reader = None
 
     def close(self):
         self.close_current_source()
 
     def read_rows(self, max_rows):
-        if not self.current_reader:
+        if not self.current_source:
             return []
 
         rows = []
         for i in xrange(min(max_rows, self.chunk_size)):
             try:
-                row = self.current_reader.next()
+                row = self.current_source.next()
                 self.num_read += 1
 
                 if 0 <= self.max_rows < self.num_read:
@@ -729,13 +833,91 @@ class ImportReader(object):
         return filter(None, rows)
 
 
-class ImportErrors(object):
+class PipeReader(object):
     """
-    A small class for managing import errors
+    A class for reading rows received on a pipe, this is used for reading input from STDIN
+    """
+    def __init__(self, inmsg, options):
+        self.inmsg = inmsg
+        self.chunk_size = options.copy['chunksize']
+        self.header = options.copy['header']
+        self.max_rows = options.copy['maxrows']
+        self.skip_rows = options.copy['skiprows']
+        self.num_read = 0
+        self.exhausted = False
+        self.num_sources = 1
+
+    def start(self):
+        pass
+
+    def read_rows(self, max_rows):
+        rows = []
+        for i in xrange(min(max_rows, self.chunk_size)):
+            row = self.inmsg.recv()
+            if row is None:
+                self.exhausted = True
+                break
+
+            self.num_read += 1
+            if 0 <= self.max_rows < self.num_read:
+                self.exhausted = True
+                break  # max rows exceeded
+
+            if self.header or self.num_read < self.skip_rows:
+                self.header = False  # skip header or initial skip_rows rows
+                continue
+
+            rows.append(row)
+
+        return rows
+
+
+class ImportProcessResult(object):
+    """
+    An object sent from ImportProcess instances to the parent import task in order to indicate progress.
+    """
+    def __init__(self, imported=0):
+        self.imported = imported
+
+
+class FeedingProcessResult(object):
+    """
+    An object sent from FeedingProcess instances to the parent import task in order to indicate progress.
+    """
+    def __init__(self, sent, reader):
+        self.sent = sent
+        self.num_sources = reader.num_sources
+        self.skip_rows = reader.skip_rows
+
+
+class ImportTaskError(object):
+    """
+    An object sent from child processes (feeder or workers) to the parent import task to indicate an error.
+    """
+    def __init__(self, name, msg, rows=None, attempts=1, final=True):
+        self.name = name
+        self.msg = msg
+        self.rows = rows if rows else []
+        self.attempts = attempts
+        self.final = final
+
+    def is_parse_error(self):
+        """
+        We treat read and parse errors as unrecoverable and we have different global counters for giving up when
+        a maximum has been reached. We consider value and type errors as parse errors as well since they
+        are typically non recoverable.
+        """
+        name = self.name
+        return name.startswith('ValueError') or name.startswith('TypeError') or \
+            name.startswith('ParseError') or name.startswith('IndexError') or name.startswith('ReadError')
+
+
+class ImportErrorHandler(object):
+    """
+    A class for managing import errors
     """
     def __init__(self, task):
         self.shell = task.shell
-        self.reader = task.reader
         self.options = task.options
         self.printmsg = task.printmsg
         self.max_attempts = self.options.copy['maxattempts']
@@ -771,42 +953,26 @@ class ImportErrors(object):
             for row in rows:
                 writer.writerow(row)
 
-    def handle_error(self, err, batch):
+    def handle_error(self, err):
         """
         Handle an error by printing the appropriate error message and incrementing the correct counter.
-        Return true if we should retry this batch, false if the error is non-recoverable
         """
         shell = self.shell
-        err = str(err)
 
-        if self.is_parse_error(err):
-            self.parse_errors += len(batch['rows'])
-            self.add_failed_rows(batch['rows'])
-            shell.printerr("Failed to import %d rows: %s -  given up without retries"
-                           % (len(batch['rows']), err))
-            return False
+        if err.is_parse_error():
+            self.parse_errors += len(err.rows)
+            self.add_failed_rows(err.rows)
+            shell.printerr("Failed to import %d rows: %s - %s,  given up without retries"
+                           % (len(err.rows), err.name, err.msg))
         else:
-            self.insert_errors += len(batch['rows'])
-            if batch['attempts'] < self.max_attempts:
-                shell.printerr("Failed to import %d rows: %s -  will retry later, attempt %d of %d"
-                               % (len(batch['rows']), err, batch['attempts'],
-                                  self.max_attempts))
-                return True
+            self.insert_errors += len(err.rows)
+            if not err.final:
+                shell.printerr("Failed to import %d rows: %s - %s,  will retry later, attempt %d of %d"
+                               % (len(err.rows), err.name, err.msg, err.attempts, self.max_attempts))
             else:
-                self.add_failed_rows(batch['rows'])
-                shell.printerr("Failed to import %d rows: %s -  given up after %d attempts"
-                               % (len(batch['rows']), err, batch['attempts']))
-                return False
-
-    @staticmethod
-    def is_parse_error(err):
-        """
-        We treat parse errors as unrecoverable and we have different global counters for giving up when
-        a maximum has been reached. We consider value and type errors as parse errors as well since they
-        are typically non recoverable.
-        """
-        return err.startswith('ValueError') or err.startswith('TypeError') or \
-            err.startswith('ParseError') or err.startswith('IndexError')
+                self.add_failed_rows(err.rows)
+                shell.printerr("Failed to import %d rows: %s - %s,  given up after %d attempts"
+                               % (len(err.rows), err.name, err.msg, err.attempts))
 
 
 class ImportTask(CopyTask):
@@ -818,22 +984,14 @@ class ImportTask(CopyTask):
         CopyTask.__init__(self, shell, ks, table, columns, fname, opts, protocol_version, config_file, 'from')
 
         options = self.options
-        self.ingest_rate = options.copy['ingestrate']
-        self.max_attempts = options.copy['maxattempts']
-        self.header = options.copy['header']
         self.skip_columns = [c.strip() for c in self.options.copy['skipcols'].split(',')]
         self.valid_columns = [c for c in self.columns if c not in self.skip_columns]
         self.table_meta = self.shell.get_table_meta(self.ks, self.table)
-        self.batch_id = 0
         self.receive_meter = RateMeter(log_fcn=self.printmsg,
                                        update_interval=options.copy['reportfrequency'],
                                        log_file=options.copy['ratefile'])
-        self.send_meter = RateMeter(log_fcn=None, update_interval=1)
-        self.reader = ImportReader(self)
-        self.import_errors = ImportErrors(self)
-        self.retries = deque([])
-        self.failed = 0
-        self.succeeded = 0
+        self.error_handler = ImportErrorHandler(self)
+        self.feeding_result = None
         self.sent = 0
 
     def make_params(self):
@@ -861,17 +1019,24 @@ class ImportTask(CopyTask):
         self.printmsg("\nStarting copy of %s.%s with columns %s." % (self.ks, self.table, self.valid_columns))
 
         try:
-            self.reader.start()
             params = self.make_params()
 
-            for i in range(self.num_processes):
-                self.processes.append(ImportProcess(params))
+            for i in range(self.num_processes - 1):
+                self.processes.append(ImportProcess(self.update_params(params, i)))
+
+            feeder = FeedingProcess(self.outmsg.channels[-1], self.inmsg.channels[-1],
+                                    self.outmsg.channels[:-1], self.fname, self.options)
+            self.processes.append(feeder)
 
-            for process in self.processes:
-                process.start()
+            self.start_processes()
+
+            pr = profile_on() if PROFILE_ON else None
 
             self.import_records()
 
+            if pr:
+                profile_off(pr, file_name='parent_profile_%d.txt' % (os.getpid(),))
+
         except Exception, exc:
             shell.printerr(str(exc))
             if shell.debug:
@@ -880,9 +1045,22 @@ class ImportTask(CopyTask):
         finally:
             self.close()
 
-    def close(self):
-        CopyTask.close(self)
-        self.reader.close()
+    def send_stdin_rows(self):
+        """
+        We need to pass stdin rows to the feeder process as it is not safe to pickle or share stdin
+        directly (in case of file the child process would close it). This is a very primitive support
+        for STDIN import in that we we won't start reporting progress until STDIN is fully consumed. I
+        think this is reasonable.
+        """
+        shell = self.shell
+
+        self.printmsg("[Use \. on a line by itself to end input]")
+        for row in shell.use_stdin_reader(prompt='[copy] ', until=r'.'):
+            self.outmsg.channels[-1].send(row)
+
+        self.outmsg.channels[-1].send(None)
+        if shell.tty:
+            print
 
     def import_records(self):
         """
@@ -890,114 +1068,137 @@ class ImportTask(CopyTask):
         Send data (batches or retries) up to the max ingest rate. If we are waiting for stuff to
         receive check the incoming queue.
         """
-        reader = self.reader
-
-        while self.has_more_to_send(reader) or self.has_more_to_receive():
-            if self.has_more_to_send(reader):
-                self.send_batches(reader)
+        if not self.fname:
+            self.send_stdin_rows()
 
-            if self.has_more_to_receive():
-                self.receive()
+        while self.feeding_result is None or self.receive_meter.total_records < self.feeding_result.sent:
+            self.receive_results()
 
-            if self.import_errors.max_exceeded() or not self.all_processes_running():
+            if self.error_handler.max_exceeded() or not self.all_processes_running():
                 break
 
-        if self.import_errors.num_rows_failed:
+        if self.error_handler.num_rows_failed:
             self.shell.printerr("Failed to process %d rows; failed rows written to %s" %
-                                (self.import_errors.num_rows_failed,
-                                 self.import_errors.err_file))
+                                (self.error_handler.num_rows_failed,
+                                 self.error_handler.err_file))
 
         if not self.all_processes_running():
             self.shell.printerr("{} child process(es) died unexpectedly, aborting"
                                 .format(self.num_processes - self.num_live_processes()))
+        else:
+            # it is only safe to write to processes if they are all running because the feeder process
+            # at the moment hangs whilst sending messages to a crashed worker process; in future
+            # we could do something about this by using a BoundedSemaphore to keep track of how many messages are
+            # queued on a pipe
+            for i, _ in enumerate(self.processes):
+                self.outmsg.channels[i].send(None)
+
+            if PROFILE_ON:
+                # allow time for worker processes to write profile results (only works if processes received
+                # the poison pill above)
+                time.sleep(5)
 
         self.printmsg("\n%d rows imported from %d files in %s (%d skipped)." %
                       (self.receive_meter.get_total_records(),
-                       self.reader.num_sources,
+                       self.feeding_result.num_sources if self.feeding_result else 0,
                        self.describe_interval(time.time() - self.time_start),
-                       self.reader.skip_rows))
-
-    def has_more_to_receive(self):
-        return (self.succeeded + self.failed) < self.sent
-
-    def has_more_to_send(self, reader):
-        return (not reader.exhausted) or self.retries
+                       self.feeding_result.skip_rows if self.feeding_result else 0))
 
     def all_processes_running(self):
-        return self.num_live_processes() == self.num_processes
+        return self.num_live_processes() == len(self.processes)
 
-    def receive(self):
-        start_time = time.time()
+    def receive_results(self):
+        """
+        Receive results from the worker processes, which will send the number of rows imported
+        or from the feeder process, which will send the number of rows sent when it has finished sending rows.
+        """
+        aggregate_result = ImportProcessResult()
+        try:
+            for result in self.inmsg.recv(timeout=0.1):
+                if isinstance(result, ImportProcessResult):
+                    aggregate_result.imported += result.imported
+                elif isinstance(result, ImportTaskError):
+                    self.error_handler.handle_error(result)
+                elif isinstance(result, FeedingProcessResult):
+                    self.feeding_result = result
+                else:
+                    raise ValueError("Unexpected result: %s" % (result,))
+        finally:
+            self.receive_meter.increment(aggregate_result.imported)
 
-        while time.time() - start_time < 0.001:
-            try:
-                batch, err = self.inmsg.get(timeout=0.00001)
 
-                if err is None:
-                    self.succeeded += batch['imported']
-                    self.receive_meter.increment(batch['imported'])
-                else:
-                    err = str(err)
+class FeedingProcess(mp.Process):
+    """
+    A process that reads from import sources and sends chunks to worker processes.
+    """
+    def __init__(self, inmsg, outmsg, worker_channels, fname, options):
+        mp.Process.__init__(self, target=self.run)
+        self.inmsg = inmsg
+        self.outmsg = outmsg
+        self.worker_channels = worker_channels
+        self.reader = FilesReader(fname, options) if fname else PipeReader(inmsg, options)
+        self.send_meter = RateMeter(log_fcn=None, update_interval=1)
+        self.ingest_rate = options.copy['ingestrate']
+        self.num_worker_processes = options.copy['numprocesses']
+        self.chunk_id = 0
+
+    def run(self):
+        pr = profile_on() if PROFILE_ON else None
 
-                    if self.import_errors.handle_error(err, batch):
-                        self.retries.append(self.reset_batch(batch))
-                    else:
-                        self.failed += len(batch['rows'])
+        self.inner_run()
 
-            except Queue.Empty:
-                pass
+        if pr:
+            profile_off(pr, file_name='feeder_profile_%d.txt' % (os.getpid(),))
 
-    def send_batches(self, reader):
+    def inner_run(self):
         """
         Send one batch per worker process to the queue unless we have exceeded the ingest rate.
         In the export case we queue everything and let the worker processes throttle using max_requests,
-        here we throttle using the ingest rate in the parent process because of memory usage concerns.
-
-        When we have finished reading the csv file, then send any retries.
+        here we throttle using the ingest rate in the feeding process because of memory usage concerns.
+        When finished we send back to the parent process the total number of rows sent.
         """
-        for _ in xrange(self.num_processes):
-            max_rows = self.ingest_rate - self.send_meter.current_record
-            if max_rows <= 0:
-                self.send_meter.maybe_update()
-                break
+        reader = self.reader
+        reader.start()
+        channels = self.worker_channels
+        sent = 0
+
+        while not reader.exhausted:
+            for ch in channels:
+                try:
+                    max_rows = self.ingest_rate - self.send_meter.current_record
+                    if max_rows <= 0:
+                        self.send_meter.maybe_update(sleep=False)
+                        continue
+
+                    rows = reader.read_rows(max_rows)
+                    if rows:
+                        sent += self.send_chunk(ch, rows)
+                except Exception, exc:
+                    self.outmsg.send(ImportTaskError(exc.__class__.__name__, exc.message))
+
+                if reader.exhausted:
+                    break
 
-            if not reader.exhausted:
-                rows = reader.read_rows(max_rows)
-                if rows:
-                    self.sent += self.send_batch(self.new_batch(rows))
-            elif self.retries:
-                batch = self.retries.popleft()
-                if len(batch['rows']) <= max_rows:
-                    self.send_batch(batch)
-                else:
-                    self.send_batch(self.split_batch(batch, batch['rows'][:max_rows]))
-                    self.retries.append(self.split_batch(batch, batch['rows'][max_rows:]))
-            else:
-                break
+        # send back to the parent process the number of rows sent to the worker processes
+        self.outmsg.send(FeedingProcessResult(sent, reader))
+
+        # wait for poison pill (None)
+        self.inmsg.recv()
 
-    def send_batch(self, batch):
-        batch['attempts'] += 1
-        num_rows = len(batch['rows'])
+    def send_chunk(self, ch, rows):
+        self.chunk_id += 1
+        num_rows = len(rows)
         self.send_meter.increment(num_rows)
-        self.outmsg.put(batch)
+        ch.send({'id': self.chunk_id, 'rows': rows, 'imported': 0, 'num_rows_sent': num_rows})
         return num_rows
 
-    def new_batch(self, rows):
-        self.batch_id += 1
-        return self.make_batch(self.batch_id, rows, 0)
-
-    @staticmethod
-    def reset_batch(batch):
-        batch['imported'] = 0
-        return batch
-
-    @staticmethod
-    def split_batch(batch, rows):
-        return ImportTask.make_batch(batch['id'], rows, batch['attempts'])
+    def close(self):
+        self.reader.close()
+        self.inmsg.close()
+        self.outmsg.close()
 
-    @staticmethod
-    def make_batch(batch_id, rows, attempts):
-        return {'id': batch_id, 'rows': rows, 'attempts': attempts, 'imported': 0}
+        for ch in self.worker_channels:
+            ch.close()
 
 
 class ChildProcess(mp.Process):
@@ -1029,6 +1230,7 @@ class ChildProcess(mp.Process):
         self.decimal_sep = options.copy['decimalsep']
         self.thousands_sep = options.copy['thousandssep']
         self.boolean_styles = options.copy['boolstyle']
+        self.max_attempts = options.copy['maxattempts']
         # Here we inject some failures for testing purposes, only if this environment variable is set
         if os.environ.get('CQLSH_COPY_TEST_FAILURES', ''):
             self.test_failures = json.loads(os.environ.get('CQLSH_COPY_TEST_FAILURES', ''))
@@ -1144,7 +1346,6 @@ class ExportProcess(ChildProcess):
         self.encoding = options.copy['encoding']
         self.float_precision = options.copy['float_precision']
         self.nullval = options.copy['nullval']
-        self.max_attempts = options.copy['maxattempts']
         self.max_requests = options.copy['maxrequests']
 
         self.hosts_to_sessions = dict()
@@ -1172,7 +1373,7 @@ class ExportProcess(ChildProcess):
                 time.sleep(0.001)  # 1 millisecond
                 continue
 
-            token_range, info = self.inmsg.get()
+            token_range, info = self.inmsg.recv()
             self.start_request(token_range, info)
 
     @staticmethod
@@ -1190,7 +1391,7 @@ class ExportProcess(ChildProcess):
     def report_error(self, err, token_range=None):
         msg = self.get_error_message(err, print_traceback=self.debug)
         self.printdebugmsg(msg)
-        self.outmsg.put((token_range, Exception(msg)))
+        self.outmsg.send((token_range, Exception(msg)))
 
     def start_request(self, token_range, info):
         """
@@ -1253,7 +1454,8 @@ class ExportProcess(ChildProcess):
             default_retry_policy=ExpBackoffRetryPolicy(self),
             compression=None,
             control_connection_timeout=self.connect_timeout,
-            connect_timeout=self.connect_timeout)
+            connect_timeout=self.connect_timeout,
+            idle_heartbeat_interval=0)
         session = ExportSession(new_cluster, self)
         self.hosts_to_sessions[host] = session
         return session
@@ -1265,7 +1467,7 @@ class ExportProcess(ChildProcess):
                 self.write_rows_to_csv(token_range, rows)
             else:
                 self.write_rows_to_csv(token_range, rows)
-                self.outmsg.put((None, None))
+                self.outmsg.send((None, None))
                 session.complete_request()
 
         def err_callback(err):
@@ -1286,7 +1488,7 @@ class ExportProcess(ChildProcess):
                 writer.writerow(map(self.format_value, row))
 
             data = (output.getvalue(), len(rows))
-            self.outmsg.put((token_range, data))
+            self.outmsg.send((token_range, data))
             output.close()
 
         except Exception, e:
@@ -1376,7 +1578,7 @@ class ImportConversion(object):
     A class for converting strings to values when importing from csv, used by ImportProcess,
     the parent.
     """
-    def __init__(self, parent, table_meta, statement):
+    def __init__(self, parent, table_meta, statement=None):
         self.ks = parent.ks
         self.table = parent.table
         self.columns = parent.valid_columns
@@ -1391,9 +1593,37 @@ class ImportConversion(object):
         self.primary_key_indexes = [self.columns.index(col.name) for col in self.table_meta.primary_key]
         self.partition_key_indexes = [self.columns.index(col.name) for col in self.table_meta.partition_key]
 
+        if statement is None:
+            self.use_prepared_statements = False
+            statement = self._get_primary_key_statement(parent, table_meta)
+        else:
+            self.use_prepared_statements = True
+
         self.proto_version = statement.protocol_version
-        self.cqltypes = dict([(c.name, c.type) for c in statement.column_metadata])
-        self.converters = dict([(c.name, self._get_converter(c.type)) for c in statement.column_metadata])
+
+        # the cql types and converters for the prepared statement, either the full statement or only the primary keys
+        self.cqltypes = [c.type for c in statement.column_metadata]
+        self.converters = [self._get_converter(c.type) for c in statement.column_metadata]
+
+        # the cql types for the entire statement, these are the same as the types above but
+        # only when using prepared statements
+        self.coltypes = [table_meta.columns[name].typestring for name in parent.valid_columns]
+        # these functions are used for non-prepared statements to protect values with quotes if required
+        self.protectors = [protect_value if t in ('ascii', 'text', 'timestamp', 'date', 'time', 'inet') else lambda v: v
+                           for t in self.coltypes]
+
+    @staticmethod
+    def _get_primary_key_statement(parent, table_meta):
+        """
+        We prepare a query statement to find out the types of the partition key columns so we can
+        route the update query to the correct replicas. As far as I understood this is the easiest
+        way to find out the types of the partition columns, we will never use this prepared statement
+        """
+        where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
+        select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(parent.ks),
+                                                         protect_name(parent.table),
+                                                         where_clause)
+        return parent.session.prepare(select_query)
 
     def _get_converter(self, cql_type):
         """
@@ -1581,27 +1811,25 @@ class ImportConversion(object):
 
         return converters.get(cql_type.typename, convert_unknown)
 
-    def get_row_values(self, row):
+    def convert_row(self, row):
         """
-        Parse the row into a list of row values to be returned
+        Convert the row into a list of parsed values if using prepared statements, else simply apply the
+        protection functions to escape values with quotes when required. Also check on the row length and
+        make sure primary partition key values aren't missing.
         """
-        def convert(n, val):
-            try:
-                return self.converters[self.columns[n]](val)
-            except Exception, e:
-                raise ParseError(e.message)
+        converters = self.converters if self.use_prepared_statements else self.protectors
 
-        ret = [None] * len(row)
-        for i, val in enumerate(row):
-            if val != self.nullval:
-                ret[i] = convert(i, val)
-            else:
-                if i in self.primary_key_indexes:
-                    raise ParseError(self.get_null_primary_key_message(i))
+        if len(row) != len(converters):
+            raise ParseError('Invalid row length %d should be %d' % (len(row), len(converters)))
 
-                ret[i] = None
+        for i in self.primary_key_indexes:
+            if row[i] == self.nullval:
+                raise ParseError(self.get_null_primary_key_message(i))
 
-        return ret
+        try:
+            return [conv(val) for conv, val in zip(converters, row)]
+        except Exception, e:
+            raise ParseError(e.message)
 
     def get_null_primary_key_message(self, idx):
         message = "Cannot insert null value for primary key column '%s'." % (self.columns[idx],)
@@ -1610,31 +1838,111 @@ class ImportConversion(object):
                        " the WITH NULL=<marker> option for COPY."
         return message
 
-    def get_row_partition_key_values(self, row):
+    def get_row_partition_key_values_fcn(self):
         """
-        Return a string composed of the partition key values, serialized and binary packed -
-        as expected by metadata.get_replicas(), see also BoundStatement.routing_key.
+        Return a function to convert a row into a string composed of the partition key values serialized
+        and binary packed (the tokens on the ring). Depending on whether we are using prepared statements, we
+        may have to convert the primary key values first, so we have two different serialize_value implementations.
+        We also return different functions depending on how many partition key indexes we have (single or multiple).
+        See also BoundStatement.routing_key.
         """
-        def serialize(n):
-            try:
-                c, v = self.columns[n], row[n]
-                if v == self.nullval:
-                    raise ParseError(self.get_null_primary_key_message(n))
-                return self.cqltypes[c].serialize(self.converters[c](v), self.proto_version)
-            except Exception, e:
-                raise ParseError(e.message)
+        def serialize_value_prepared(n, v):
+            return self.cqltypes[n].serialize(v, self.proto_version)
+
+        def serialize_value_not_prepared(n, v):
+            return self.cqltypes[n].serialize(self.converters[n](v), self.proto_version)
 
         partition_key_indexes = self.partition_key_indexes
-        if len(partition_key_indexes) == 1:
-            return serialize(partition_key_indexes[0])
-        else:
+        serialize = serialize_value_prepared if self.use_prepared_statements else serialize_value_not_prepared
+
+        def serialize_row_single(row):
+            return serialize(partition_key_indexes[0], row[partition_key_indexes[0]])
+
+        def serialize_row_multiple(row):
             pk_values = []
             for i in partition_key_indexes:
-                val = serialize(i)
+                val = serialize(i, row[i])
                 l = len(val)
                 pk_values.append(struct.pack(">H%dsB" % l, l, val, 0))
             return b"".join(pk_values)
 
+        if len(partition_key_indexes) == 1:
+            return serialize_row_single
+        return serialize_row_multiple
+
+
+class TokenMap(object):
+    """
+    A wrapper around the metadata token map to speed things up by caching ring token *values* and
+    replicas. It is very important that we use the token values, which are primitive types, rather
+    than the tokens classes when calling bisect_right() in split_batches(). If we use primitive values,
+    the bisect is done in compiled code whilst with token classes each comparison requires a call
+    into the interpreter to perform the cmp operation defined in Python. A simple test with 1 million bisect
+    operations on an array of 2048 tokens was done in 0.37 seconds with primitives and 2.25 seconds with
+    token classes. This is significant for large datasets because we need to do a bisect for each single row,
+    and if VNODES are used, the size of the token map can get quite large too.
+    """
+    def __init__(self, ks, hostname, local_dc, session):
+
+        self.ks = ks
+        self.hostname = hostname
+        self.local_dc = local_dc
+        self.metadata = session.cluster.metadata
+
+        self._initialize_ring()
+
+        # Note that refresh metadata is disabled by default and we currenlty do not intercept it
+        # If hosts are added, removed or moved during a COPY operation our token map is no longer optimal
+        # However we can cope with hosts going down and up since we filter for replicas that are up when
+        # making each batch
+
+    def _initialize_ring(self):
+        token_map = self.metadata.token_map
+        if token_map is None:
+            self.ring = [0]
+            self.replicas = [(self.metadata.get_host(self.hostname),)]
+            self.pk_to_token_value = lambda pk: 0
+            return
+
+        token_map.rebuild_keyspace(self.ks, build_if_absent=True)
+        tokens_to_hosts = token_map.tokens_to_hosts_by_ks.get(self.ks, None)
+        from_key = token_map.token_class.from_key
+
+        self.ring = [token.value for token in token_map.ring]
+        self.replicas = [tuple(tokens_to_hosts[token]) for token in token_map.ring]
+        self.pk_to_token_value = lambda pk: from_key(pk).value
+
+    @staticmethod
+    def get_ring_pos(ring, val):
+        idx = bisect_right(ring, val)
+        return idx if idx < len(ring) else 0
+
+    def filter_replicas(self, hosts):
+        shuffled = tuple(sorted(hosts, key=lambda k: random.random()))
+        return filter(lambda r: r.is_up and r.datacenter == self.local_dc, shuffled) if hosts else ()
+
+
+class FastTokenAwarePolicy(DCAwareRoundRobinPolicy):
+    """
+    Send to any replicas attached to the query, or else fall back to DCAwareRoundRobinPolicy
+    """
+
+    def __init__(self, local_dc='', used_hosts_per_remote_dc=0):
+        DCAwareRoundRobinPolicy.__init__(self, local_dc, used_hosts_per_remote_dc)
+
+    def make_query_plan(self, working_keyspace=None, query=None):
+        """
+        Extend TokenAwarePolicy.make_query_plan() so that we choose the same replicas in preference
+        and most importantly we avoid repeating the (slow) bisect
+        """
+        replicas = query.replicas if hasattr(query, 'replicas') else []
+        for r in replicas:
+            yield r
+
+        for r in DCAwareRoundRobinPolicy.make_query_plan(self, working_keyspace, query):
+            if r not in replicas:
+                yield r
+
 
 class ImportProcess(ChildProcess):
 
@@ -1650,7 +1958,12 @@ class ImportProcess(ChildProcess):
         self.max_attempts = options.copy['maxattempts']
         self.min_batch_size = options.copy['minbatchsize']
         self.max_batch_size = options.copy['maxbatchsize']
+        self.use_prepared_statements = options.copy['preparedstatements']
+        self.dialect_options = options.dialect
         self._session = None
+        self.query = None
+        self.conv = None
+        self.make_statement = None
 
     @property
     def session(self):
@@ -1661,12 +1974,13 @@ class ImportProcess(ChildProcess):
                 cql_version=self.cql_version,
                 protocol_version=self.protocol_version,
                 auth_provider=self.auth_provider,
-                load_balancing_policy=TokenAwarePolicy(DCAwareRoundRobinPolicy(local_dc=self.local_dc)),
+                load_balancing_policy=FastTokenAwarePolicy(local_dc=self.local_dc),
                 ssl_options=ssl_settings(self.hostname, self.config_file) if self.ssl else None,
                 default_retry_policy=ExpBackoffRetryPolicy(self),
                 compression=None,
                 control_connection_timeout=self.connect_timeout,
-                connect_timeout=self.connect_timeout)
+                connect_timeout=self.connect_timeout,
+                idle_heartbeat_interval=0)
 
             self._session = cluster.connect(self.ks)
             self._session.default_timeout = None
@@ -1674,13 +1988,12 @@ class ImportProcess(ChildProcess):
 
     def run(self):
         try:
-            table_meta = self.session.cluster.metadata.keyspaces[self.ks].tables[self.table]
-            is_counter = ("counter" in [table_meta.columns[name].typestring for name in self.valid_columns])
+            pr = profile_on() if PROFILE_ON else None
 
-            if is_counter:
-                self.run_counter(table_meta)
-            else:
-                self.run_normal(table_meta)
+            self.inner_run(*self.make_params())
+
+            if pr:
+                profile_off(pr, file_name='worker_profile_%d.txt' % (os.getpid(),))
 
         except Exception, exc:
             if self.debug:
@@ -1694,67 +2007,88 @@ class ImportProcess(ChildProcess):
             self._session.cluster.shutdown()
         ChildProcess.close(self)
 
-    def run_counter(self, table_meta):
-        """
-        Main run method for tables that contain counter columns.
-        """
-        query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
-
-        # We prepare a query statement to find out the types of the partition key columns so we can
-        # route the update query to the correct replicas. As far as I understood this is the easiest
-        # way to find out the types of the partition columns, we will never use this prepared statement
-        where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
-        select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(self.ks), protect_name(self.table), where_clause)
-        conv = ImportConversion(self, table_meta, self.session.prepare(select_query))
-
-        while True:
-            batch = self.inmsg.get()
-            try:
-                for b in self.split_batches(batch, conv):
-                    self.send_counter_batch(query, conv, b)
+    def make_params(self):
+        metadata = self.session.cluster.metadata
+        table_meta = metadata.keyspaces[self.ks].tables[self.table]
+
+        prepared_statement = None
+        is_counter = ("counter" in [table_meta.columns[name].typestring for name in self.valid_columns])
+        if is_counter:
+            query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
+            make_statement = self.wrap_make_statement(self.make_counter_batch_statement)
+        elif self.use_prepared_statements:
+            query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
+                                                            protect_name(self.table),
+                                                            ', '.join(protect_names(self.valid_columns),),
+                                                            ', '.join(['?' for _ in self.valid_columns]))
+
+            query = self.session.prepare(query)
+            query.consistency_level = self.consistency_level
+            prepared_statement = query
+            make_statement = self.wrap_make_statement(self.make_prepared_batch_statement)
+        else:
+            query = 'INSERT INTO %s.%s (%s) VALUES (%%s)' % (protect_name(self.ks),
+                                                             protect_name(self.table),
+                                                             ', '.join(protect_names(self.valid_columns),))
+            make_statement = self.wrap_make_statement(self.make_non_prepared_batch_statement)
 
-            except Exception, exc:
-                self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                if self.debug:
-                    traceback.print_exc(exc)
+        conv = ImportConversion(self, table_meta, prepared_statement)
+        tm = TokenMap(self.ks, self.hostname, self.local_dc, self.session)
+        return query, conv, tm, make_statement
 
-    def run_normal(self, table_meta):
+    def inner_run(self, query, conv, tm, make_statement):
         """
-        Main run method for normal tables, i.e. tables that do not contain counter columns.
+        Main run method. Note that we bind self methods that are called inside loops
+        for performance reasons.
         """
-        query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
-                                                        protect_name(self.table),
-                                                        ', '.join(protect_names(self.valid_columns),),
-                                                        ', '.join(['?' for _ in self.valid_columns]))
+        self.query = query
+        self.conv = conv
+        self.make_statement = make_statement
 
-        query_statement = self.session.prepare(query)
-        query_statement.consistency_level = self.consistency_level
-        conv = ImportConversion(self, table_meta, query_statement)
+        convert_rows = self.convert_rows
+        split_into_batches = self.split_into_batches
+        result_callback = self.result_callback
+        err_callback = self.err_callback
+        session = self.session
 
         while True:
-            batch = self.inmsg.get()
+            chunk = self.inmsg.recv()
+            if chunk is None:
+                break
+
             try:
-                for b in self.split_batches(batch, conv):
-                    self.send_normal_batch(conv, query_statement, b)
+                chunk['rows'] = convert_rows(conv, chunk)
+                for replicas, batch in split_into_batches(chunk, conv, tm):
+                    statement = make_statement(query, conv, chunk, batch, replicas)
+                    future = session.execute_async(statement)
+                    future.add_callbacks(callback=result_callback, callback_args=(batch, chunk),
+                                         errback=err_callback, errback_args=(batch, chunk, replicas))
 
             except Exception, exc:
-                self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                if self.debug:
-                    traceback.print_exc(exc)
+                self.report_error(exc, chunk, chunk['rows'])
 
-    def send_counter_batch(self, query_text, conv, batch):
-        if self.test_failures and self.maybe_inject_failures(batch):
-            return
+    def wrap_make_statement(self, inner_make_statement):
+        def make_statement(query, conv, chunk, batch, replicas):
+            try:
+                return inner_make_statement(query, conv, batch, replicas)
+            except Exception, exc:
+                print "Failed to make batch statement: {}".format(exc)
+                self.report_error(exc, chunk, batch['rows'])
+                return None
 
-        error_rows = []
-        batch_statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+        def make_statement_with_failures(query, conv, chunk, batch, replicas):
+            failed_batch = self.maybe_inject_failures(batch)
+            if failed_batch:
+                return failed_batch
+            return make_statement(query, conv, chunk, batch, replicas)
 
-        for r in batch['rows']:
-            row = self.filter_row_values(r)
-            if len(row) != len(self.valid_columns):
-                error_rows.append(row)
-                continue
+        return make_statement_with_failures if self.test_failures else make_statement
 
+    def make_counter_batch_statement(self, query, conv, batch, replicas):
+        statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+        statement.replicas = replicas
+        statement.keyspace = self.ks
+        for row in batch['rows']:
             where_clause = []
             set_clause = []
             for i, value in enumerate(row):
@@ -1763,65 +2097,61 @@ class ImportProcess(ChildProcess):
                 else:
                     set_clause.append("%s=%s+%s" % (self.valid_columns[i], self.valid_columns[i], value))
 
-            full_query_text = query_text % (','.join(set_clause), ' AND '.join(where_clause))
-            batch_statement.add(full_query_text)
-
-        self.execute_statement(batch_statement, batch)
+            full_query_text = query % (','.join(set_clause), ' AND '.join(where_clause))
+            statement.add(full_query_text)
+        return statement
 
-        if error_rows:
-            self.outmsg.put((ImportTask.split_batch(batch, error_rows),
-                            '%s - %s' % (ParseError.__name__, "Failed to parse one or more rows")))
+    def make_prepared_batch_statement(self, query, _, batch, replicas):
+        """
+        Return a batch statement. This is an optimized version of:
 
-    def send_normal_batch(self, conv, query_statement, batch):
-        if self.test_failures and self.maybe_inject_failures(batch):
-            return
+            statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+            for row in batch['rows']:
+                statement.add(query, row)
 
-        good_rows, converted_rows, errors = self.convert_rows(conv, batch['rows'])
+        We could optimize further by removing bound_statements altogether but we'd have to duplicate much
+        more driver's code (BoundStatement.bind()).
+        """
+        statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+        statement.replicas = replicas
+        statement.keyspace = self.ks
+        statement._statements_and_parameters = [(True, query.query_id, query.bind(r).values) for r in batch['rows']]
+        return statement
 
-        if converted_rows:
-            try:
-                statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
-                for row in converted_rows:
-                    statement.add(query_statement, row)
-                self.execute_statement(statement, ImportTask.split_batch(batch, good_rows))
-            except Exception, exc:
-                self.err_callback(exc, ImportTask.split_batch(batch, good_rows))
+    def make_non_prepared_batch_statement(self, query, _, batch, replicas):
+        statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+        statement.replicas = replicas
+        statement.keyspace = self.ks
+        statement._statements_and_parameters = [(False, query % (','.join(r),), ()) for r in batch['rows']]
+        return statement
 
-        if errors:
-            for msg, rows in errors.iteritems():
-                self.outmsg.put((ImportTask.split_batch(batch, rows),
-                                '%s - %s' % (ParseError.__name__, msg)))
-
-    def convert_rows(self, conv, rows):
+    def convert_rows(self, conv, chunk):
         """
-        Try to convert each row. If conversion is OK then add the converted result to converted_rows
-        and the original string to good_rows. Else add the original string to error_rows. Return the three
-        arrays.
+        Return converted rows and report any errors during conversion.
         """
-        good_rows = []
-        errors = defaultdict(list)
-        converted_rows = []
+        def filter_row_values(row):
+            return [v for i, v in enumerate(row) if i not in self.skip_column_indexes]
 
-        for r in rows:
-            row = self.filter_row_values(r)
-            if len(row) != len(self.valid_columns):
-                msg = 'Invalid row length %d should be %d' % (len(row), len(self.valid_columns))
-                errors[msg].append(row)
-                continue
+        if self.skip_column_indexes:
+            rows = [filter_row_values(r) for r in list(csv.reader(chunk['rows'], **self.dialect_options))]
+        else:
+            rows = list(csv.reader(chunk['rows'], **self.dialect_options))
 
-            try:
-                converted_rows.append(conv.get_row_values(row))
-                good_rows.append(row)
-            except ParseError, err:
-                errors[err.message].append(row)
+        errors = defaultdict(list)
 
-        return good_rows, converted_rows, errors
+        def convert_row(r):
+            try:
+                return conv.convert_row(r)
+            except Exception, err:
+                errors[err.message].append(r)
+                return None
 
-    def filter_row_values(self, row):
-        if not self.skip_column_indexes:
-            return row
+        converted_rows = filter(None, [convert_row(r) for r in rows])
 
-        return [v for i, v in enumerate(row) if i not in self.skip_column_indexes]
+        if errors:
+            for msg, rows in errors.iteritems():
+                self.report_error(ParseError(msg), chunk, rows)
+        return converted_rows
 
     def maybe_inject_failures(self, batch):
         """
@@ -1836,86 +2166,94 @@ class ImportProcess(ChildProcess):
                 if batch['attempts'] < failing_batch['failures']:
                     statement = SimpleStatement("INSERT INTO badtable (a, b) VALUES (1, 2)",
                                                 consistency_level=self.consistency_level)
-                    self.execute_statement(statement, batch)
-                    return True
+                    return statement
 
         if 'exit_batch' in self.test_failures:
             exit_batch = self.test_failures['exit_batch']
             if exit_batch['id'] == batch['id']:
                 sys.exit(1)
 
-        return False  # carry on as normal
+        return None  # carry on as normal
 
-    def execute_statement(self, statement, batch):
-        future = self.session.execute_async(statement)
-        future.add_callbacks(callback=self.result_callback, callback_args=(batch, ),
-                             errback=self.err_callback, errback_args=(batch, ))
+    @staticmethod
+    def make_batch(batch_id, rows, attempts=1):
+        return {'id': batch_id, 'rows': rows, 'attempts': attempts}
 
-    def split_batches(self, batch, conv):
+    def split_into_batches(self, chunk, conv, tm):
         """
-        Batch rows by partition key, if there are at least min_batch_size (2)
-        rows with the same partition key. These batches can be as big as they want
-        since this translates to a single insert operation server side.
-
-        If there are less than min_batch_size rows for a partition, work out the
-        first replica for this partition and add the rows to replica left-over rows.
-
-        Then batch the left-overs of each replica up to max_batch_size.
+        Batch rows by ring position or replica.
+        If there are at least min_batch_size rows for a ring position then split these rows into
+        groups of max_batch_size and send a batch for each group, using all replicas for this ring position.
+        Otherwise, we are forced to batch by replica, and here unfortunately we can only choose one replica to
+        guarantee common replicas across partition keys. We are typically able
+        to batch by ring position for small clusters or when VNODES are not used. For large clusters with VNODES
+        it may not be possible, in this case it helps to increase the CHUNK SIZE but up to a limit, otherwise
+        we may choke the cluster.
         """
-        rows_by_pk = defaultdict(list)
+
+        rows_by_ring_pos = defaultdict(list)
         errors = defaultdict(list)
 
-        for row in batch['rows']:
+        min_batch_size = self.min_batch_size
+        max_batch_size = self.max_batch_size
+        ring = tm.ring
+
+        get_row_partition_key_values = conv.get_row_partition_key_values_fcn()
+        pk_to_token_value = tm.pk_to_token_value
+        get_ring_pos = tm.get_ring_pos
+        make_batch = self.make_batch
+
+        for row in chunk['rows']:
             try:
-                pk = conv.get_row_partition_key_values(row)
-                rows_by_pk[pk].append(row)
-            except ParseError, e:
+                pk = get_row_partition_key_values(row)
+                rows_by_ring_pos[get_ring_pos(ring, pk_to_token_value(pk))].append(row)
+            except Exception, e:
                 errors[e.message].append(row)
 
         if errors:
             for msg, rows in errors.iteritems():
-                self.outmsg.put((ImportTask.split_batch(batch, rows),
-                                 '%s - %s' % (ParseError.__name__, msg)))
+                self.report_error(ParseError(msg), chunk, rows)
 
+        replicas = tm.replicas
+        filter_replicas = tm.filter_replicas
         rows_by_replica = defaultdict(list)
-        for pk, rows in rows_by_pk.iteritems():
-            if len(rows) >= self.min_batch_size:
-                yield ImportTask.make_batch(batch['id'], rows, batch['attempts'])
+        for ring_pos, rows in rows_by_ring_pos.iteritems():
+            if len(rows) > min_batch_size:
+                for i in xrange(0, len(rows), max_batch_size):
+                    yield filter_replicas(replicas[ring_pos]), make_batch(chunk['id'], rows[i:i + max_batch_size])
             else:
-                replica = self.get_replica(pk)
-                rows_by_replica[replica].extend(rows)
-
-        for replica, rows in rows_by_replica.iteritems():
-            for b in self.batches(rows, batch):
-                yield b
-
-    def get_replica(self, pk):
-        """
-        Return the first replica or the host we are already connected to if there are no local
-        replicas that are up. We always use the first replica to match the replica chosen by the driver
-        TAR, see TokenAwarePolicy.make_query_plan().
-        """
-        metadata = self.session.cluster.metadata
-        replicas = filter(lambda r: r.is_up and r.datacenter == self.local_dc, metadata.get_replicas(self.ks, pk))
-        ret = replicas[0].address if len(replicas) > 0 else self.hostname
-        return ret
-
-    def batches(self, rows, batch):
-        """
-        Split rows into batches of max_batch_size
-        """
-        for i in xrange(0, len(rows), self.max_batch_size):
-            yield ImportTask.make_batch(batch['id'], rows[i:i + self.max_batch_size], batch['attempts'])
-
-    def result_callback(self, _, batch):
-        batch['imported'] = len(batch['rows'])
-        batch['rows'] = []  # no need to resend these, just send the count in 'imported'
-        self.outmsg.put((batch, None))
-
-    def err_callback(self, response, batch):
-        self.outmsg.put((batch, '%s - %s' % (response.__class__.__name__, response.message)))
+                # select only the first valid replica to guarantee more overlap or none at all
+                rows_by_replica[filter_replicas(replicas[ring_pos])[:1]].extend(rows)
+
+        # Now send the batches by replica
+        for replicas, rows in rows_by_replica.iteritems():
+            for i in xrange(0, len(rows), max_batch_size):
+                yield replicas, make_batch(chunk['id'], rows[i:i + max_batch_size])
+
+    def result_callback(self, _, batch, chunk):
+        self.update_chunk(batch['rows'], chunk)
+
+    def err_callback(self, response, batch, chunk, replicas):
+        err_is_final = batch['attempts'] >= self.max_attempts
+        self.report_error(response, chunk, batch['rows'], batch['attempts'], err_is_final)
+        if not err_is_final:
+            batch['attempts'] += 1
+            statement = self.make_statement(self.query, self.conv, chunk, batch, replicas)
+            future = self.session.execute_async(statement)
+            future.add_callbacks(callback=self.result_callback, callback_args=(batch, chunk),
+                                 errback=self.err_callback, errback_args=(batch, chunk, replicas))
+
+    def report_error(self, err, chunk, rows=None, attempts=1, final=True):
         if self.debug:
-            traceback.print_exc(response)
+            traceback.print_exc(err)
+        self.outmsg.send(ImportTaskError(err.__class__.__name__, err.message, rows, attempts, final))
+        if final:
+            self.update_chunk(rows, chunk)
+
+    def update_chunk(self, rows, chunk):
+        chunk['imported'] += len(rows)
+        if chunk['imported'] == chunk['num_rows_sent']:
+            self.outmsg.send(ImportProcessResult(chunk['num_rows_sent']))
 
 
 class RateMeter(object):
@@ -1937,11 +2275,19 @@ class RateMeter(object):
         self.current_record += n
         self.maybe_update()
 
-    def maybe_update(self):
+    def maybe_update(self, sleep=False):
+        if self.current_record == 0:
+            return
+
         new_checkpoint_time = time.time()
-        if new_checkpoint_time - self.last_checkpoint_time >= self.update_interval:
+        time_difference = new_checkpoint_time - self.last_checkpoint_time
+        if time_difference >= self.update_interval:
             self.update(new_checkpoint_time)
             self.log_message()
+        elif sleep:
+            remaining_time = time_difference - self.update_interval
+            if remaining_time > 0.000001:
+                time.sleep(remaining_time)
 
     def update(self, new_checkpoint_time):
         time_difference = new_checkpoint_time - self.last_checkpoint_time

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/pylib/cqlshlib/util.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/util.py b/pylib/cqlshlib/util.py
index 281aad6..3ee128d 100644
--- a/pylib/cqlshlib/util.py
+++ b/pylib/cqlshlib/util.py
@@ -23,6 +23,12 @@ from itertools import izip
 from datetime import timedelta, tzinfo
 from StringIO import StringIO
 
+try:
+    from line_profiler import LineProfiler
+    HAS_LINE_PROFILER = True
+except ImportError:
+    HAS_LINE_PROFILER = False
+
 ZERO = timedelta(0)
 
 
@@ -126,18 +132,35 @@ def get_file_encoding_bomsize(filename):
     else:
         file_encoding, size = "utf-8", 0
 
-    return (file_encoding, size)
+    return file_encoding, size
+
 
+def profile_on(fcn_names=None):
+    if fcn_names and HAS_LINE_PROFILER:
+        pr = LineProfiler()
+        for fcn_name in fcn_names:
+            pr.add_function(fcn_name)
+        pr.enable()
+        return pr
 
-def profile_on():
     pr = cProfile.Profile()
     pr.enable()
     return pr
 
 
-def profile_off(pr):
+def profile_off(pr, file_name):
     pr.disable()
     s = StringIO()
-    ps = pstats.Stats(pr, stream=s).sort_stats('cumulative')
-    ps.print_stats()
-    print s.getvalue()
+
+    if HAS_LINE_PROFILER and isinstance(pr, LineProfiler):
+        pr.print_stats(s)
+    else:
+        ps = pstats.Stats(pr, stream=s).sort_stats('cumulative')
+        ps.print_stats()
+
+    ret = s.getvalue()
+    if file_name:
+        with open(file_name, 'w') as f:
+            print "Writing to %s\n" % (f.name, )
+            f.write(ret)
+    return ret

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/pylib/setup.py
----------------------------------------------------------------------
diff --git a/pylib/setup.py b/pylib/setup.py
index 704d077..3654502 100755
--- a/pylib/setup.py
+++ b/pylib/setup.py
@@ -16,9 +16,11 @@
 # limitations under the License.
 
 from distutils.core import setup
+from Cython.Build import cythonize
 
 setup(
     name="cassandra-pylib",
     description="Cassandra Python Libraries",
     packages=["cqlshlib"],
+    ext_modules=cythonize("cqlshlib/copyutil.py"),
 )


[15/23] cassandra git commit: Merge commit 'c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc' into cassandra-2.2

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b74ffeaf/pylib/cqlshlib/copyutil.py
----------------------------------------------------------------------
diff --cc pylib/cqlshlib/copyutil.py
index aeb2d0b,cd03765..2755dd5
--- a/pylib/cqlshlib/copyutil.py
+++ b/pylib/cqlshlib/copyutil.py
@@@ -45,9 -51,13 +51,13 @@@ from cassandra.util import Date, Tim
  
  from cql3handling import CqlRuleSet
  from displaying import NO_COLOR_MAP
 -from formatting import format_value_default, EMPTY, get_formatter
 +from formatting import format_value_default, DateTimeFormat, EMPTY, get_formatter
  from sslhandling import ssl_settings
  
+ PROFILE_ON = False
+ STRACE_ON = False
+ IS_LINUX = platform.system() == 'Linux'
+ 
  CopyOptions = namedtuple('CopyOptions', 'copy dialect unrecognized')
  
  
@@@ -164,13 -252,12 +252,13 @@@ class CopyTask(object)
          # in the page size or 10 seconds if pagesize is smaller
          copy_options['pagetimeout'] = int(opts.pop('pagetimeout', max(10, 10 * (copy_options['pagesize'] / 1000))))
          copy_options['maxattempts'] = int(opts.pop('maxattempts', 5))
 -        copy_options['dtformats'] = opts.pop('datetimeformat', shell.display_time_format)
 +        copy_options['dtformats'] = DateTimeFormat(opts.pop('datetimeformat', shell.display_timestamp_format),
 +                                                   shell.display_date_format, shell.display_nanotime_format)
          copy_options['float_precision'] = shell.display_float_precision
-         copy_options['chunksize'] = int(opts.pop('chunksize', 1000))
+         copy_options['chunksize'] = int(opts.pop('chunksize', 5000))
          copy_options['ingestrate'] = int(opts.pop('ingestrate', 100000))
          copy_options['maxbatchsize'] = int(opts.pop('maxbatchsize', 20))
-         copy_options['minbatchsize'] = int(opts.pop('minbatchsize', 2))
+         copy_options['minbatchsize'] = int(opts.pop('minbatchsize', 10))
          copy_options['reportfrequency'] = float(opts.pop('reportfrequency', 0.25))
          copy_options['consistencylevel'] = shell.consistency_level
          copy_options['decimalsep'] = opts.pop('decimalsep', '.')
@@@ -1392,9 -1593,37 +1594,37 @@@ class ImportConversion(object)
          self.primary_key_indexes = [self.columns.index(col.name) for col in self.table_meta.primary_key]
          self.partition_key_indexes = [self.columns.index(col.name) for col in self.table_meta.partition_key]
  
+         if statement is None:
+             self.use_prepared_statements = False
+             statement = self._get_primary_key_statement(parent, table_meta)
+         else:
+             self.use_prepared_statements = True
+ 
          self.proto_version = statement.protocol_version
-         self.cqltypes = dict([(c.name, c.type) for c in statement.column_metadata])
-         self.converters = dict([(c.name, self._get_converter(c.type)) for c in statement.column_metadata])
+ 
+         # the cql types and converters for the prepared statement, either the full statement or only the primary keys
+         self.cqltypes = [c.type for c in statement.column_metadata]
+         self.converters = [self._get_converter(c.type) for c in statement.column_metadata]
+ 
+         # the cql types for the entire statement, these are the same as the types above but
+         # only when using prepared statements
 -        self.coltypes = [table_meta.columns[name].typestring for name in parent.valid_columns]
++        self.coltypes = [table_meta.columns[name].cql_type for name in parent.valid_columns]
+         # these functions are used for non-prepared statements to protect values with quotes if required
+         self.protectors = [protect_value if t in ('ascii', 'text', 'timestamp', 'date', 'time', 'inet') else lambda v: v
+                            for t in self.coltypes]
+ 
+     @staticmethod
+     def _get_primary_key_statement(parent, table_meta):
+         """
+         We prepare a query statement to find out the types of the partition key columns so we can
+         route the update query to the correct replicas. As far as I understood this is the easiest
+         way to find out the types of the partition columns, we will never use this prepared statement
+         """
+         where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
+         select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(parent.ks),
+                                                          protect_name(parent.table),
+                                                          where_clause)
+         return parent.session.prepare(select_query)
  
      def _get_converter(self, cql_type):
          """
@@@ -1695,67 -2007,88 +2008,88 @@@ class ImportProcess(ChildProcess)
              self._session.cluster.shutdown()
          ChildProcess.close(self)
  
-     def run_counter(self, table_meta):
-         """
-         Main run method for tables that contain counter columns.
-         """
-         query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
- 
-         # We prepare a query statement to find out the types of the partition key columns so we can
-         # route the update query to the correct replicas. As far as I understood this is the easiest
-         # way to find out the types of the partition columns, we will never use this prepared statement
-         where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
-         select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(self.ks), protect_name(self.table), where_clause)
-         conv = ImportConversion(self, table_meta, self.session.prepare(select_query))
- 
-         while True:
-             batch = self.inmsg.get()
-             try:
-                 for b in self.split_batches(batch, conv):
-                     self.send_counter_batch(query, conv, b)
+     def make_params(self):
+         metadata = self.session.cluster.metadata
+         table_meta = metadata.keyspaces[self.ks].tables[self.table]
+ 
+         prepared_statement = None
 -        is_counter = ("counter" in [table_meta.columns[name].typestring for name in self.valid_columns])
++        is_counter = ("counter" in [table_meta.columns[name].cql_type for name in self.valid_columns])
+         if is_counter:
+             query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
+             make_statement = self.wrap_make_statement(self.make_counter_batch_statement)
+         elif self.use_prepared_statements:
+             query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
+                                                             protect_name(self.table),
+                                                             ', '.join(protect_names(self.valid_columns),),
+                                                             ', '.join(['?' for _ in self.valid_columns]))
+ 
+             query = self.session.prepare(query)
+             query.consistency_level = self.consistency_level
+             prepared_statement = query
+             make_statement = self.wrap_make_statement(self.make_prepared_batch_statement)
+         else:
+             query = 'INSERT INTO %s.%s (%s) VALUES (%%s)' % (protect_name(self.ks),
+                                                              protect_name(self.table),
+                                                              ', '.join(protect_names(self.valid_columns),))
+             make_statement = self.wrap_make_statement(self.make_non_prepared_batch_statement)
  
-             except Exception, exc:
-                 self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                 if self.debug:
-                     traceback.print_exc(exc)
+         conv = ImportConversion(self, table_meta, prepared_statement)
+         tm = TokenMap(self.ks, self.hostname, self.local_dc, self.session)
+         return query, conv, tm, make_statement
  
-     def run_normal(self, table_meta):
+     def inner_run(self, query, conv, tm, make_statement):
          """
-         Main run method for normal tables, i.e. tables that do not contain counter columns.
+         Main run method. Note that we bind self methods that are called inside loops
+         for performance reasons.
          """
-         query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
-                                                         protect_name(self.table),
-                                                         ', '.join(protect_names(self.valid_columns),),
-                                                         ', '.join(['?' for _ in self.valid_columns]))
+         self.query = query
+         self.conv = conv
+         self.make_statement = make_statement
  
-         query_statement = self.session.prepare(query)
-         query_statement.consistency_level = self.consistency_level
-         conv = ImportConversion(self, table_meta, query_statement)
+         convert_rows = self.convert_rows
+         split_into_batches = self.split_into_batches
+         result_callback = self.result_callback
+         err_callback = self.err_callback
+         session = self.session
  
          while True:
-             batch = self.inmsg.get()
+             chunk = self.inmsg.recv()
+             if chunk is None:
+                 break
+ 
              try:
-                 for b in self.split_batches(batch, conv):
-                     self.send_normal_batch(conv, query_statement, b)
+                 chunk['rows'] = convert_rows(conv, chunk)
+                 for replicas, batch in split_into_batches(chunk, conv, tm):
+                     statement = make_statement(query, conv, chunk, batch, replicas)
+                     future = session.execute_async(statement)
+                     future.add_callbacks(callback=result_callback, callback_args=(batch, chunk),
+                                          errback=err_callback, errback_args=(batch, chunk, replicas))
  
              except Exception, exc:
-                 self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                 if self.debug:
-                     traceback.print_exc(exc)
+                 self.report_error(exc, chunk, chunk['rows'])
  
-     def send_counter_batch(self, query_text, conv, batch):
-         if self.test_failures and self.maybe_inject_failures(batch):
-             return
+     def wrap_make_statement(self, inner_make_statement):
+         def make_statement(query, conv, chunk, batch, replicas):
+             try:
+                 return inner_make_statement(query, conv, batch, replicas)
+             except Exception, exc:
+                 print "Failed to make batch statement: {}".format(exc)
+                 self.report_error(exc, chunk, batch['rows'])
+                 return None
  
-         error_rows = []
-         batch_statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+         def make_statement_with_failures(query, conv, chunk, batch, replicas):
+             failed_batch = self.maybe_inject_failures(batch)
+             if failed_batch:
+                 return failed_batch
+             return make_statement(query, conv, chunk, batch, replicas)
  
-         for r in batch['rows']:
-             row = self.filter_row_values(r)
-             if len(row) != len(self.valid_columns):
-                 error_rows.append(row)
-                 continue
+         return make_statement_with_failures if self.test_failures else make_statement
  
+     def make_counter_batch_statement(self, query, conv, batch, replicas):
+         statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+         statement.replicas = replicas
+         statement.keyspace = self.ks
+         for row in batch['rows']:
              where_clause = []
              set_clause = []
              for i, value in enumerate(row):


[19/23] cassandra git commit: Merge commit 'b74ffeafd2e08a7669013d6b4ba87980e1ad5379' into cassandra-3.0

Posted by sl...@apache.org.
Merge commit 'b74ffeafd2e08a7669013d6b4ba87980e1ad5379' into cassandra-3.0


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/49c616cf
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/49c616cf
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/49c616cf

Branch: refs/heads/cassandra-3.5
Commit: 49c616cf0fc9c8d4649e2ec71a07fb9fd7831318
Parents: 34b07a7 b74ffea
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue Mar 8 10:23:48 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Mar 8 10:24:26 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                |    2 +-
 bin/cqlsh.py               |   33 +-
 pylib/cqlshlib/copyutil.py | 1168 +++++++++++++++++++++++++--------------
 pylib/cqlshlib/util.py     |   35 +-
 pylib/setup.py             |    2 +
 5 files changed, 821 insertions(+), 419 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/49c616cf/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 38bf2ee,7b67cdc..5b92143
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,39 -1,11 +1,39 @@@
 -2.2.6
 +3.0.5
 + * Fix filtering on non-primary key columns for queries without index (CASSANDRA-6377)
 + * Fix sstableloader fail when using materialized view (CASSANDRA-11275)
 +Merged from 2.2:
   * Fix AE in nodetool cfstats (backport CASSANDRA-10859) (CASSANDRA-11297)
   * Unresolved hostname leads to replace being ignored (CASSANDRA-11210)
 - * Fix filtering on non-primary key columns for thrift static column families
 -   (CASSANDRA-6377)
   * Only log yaml config once, at startup (CASSANDRA-11217)
 - * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
   * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
 +Merged from 2.1:
 + * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
- 
++ * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
 +
 +3.0.4
 + * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
 + * MV should only query complex columns included in the view (CASSANDRA-11069)
 + * Failed aggregate creation breaks server permanently (CASSANDRA-11064)
 + * Add sstabledump tool (CASSANDRA-7464)
 + * Introduce backpressure for hints (CASSANDRA-10972)
 + * Fix ClusteringPrefix not being able to read tombstone range boundaries (CASSANDRA-11158)
 + * Prevent logging in sandboxed state (CASSANDRA-11033)
 + * Disallow drop/alter operations of UDTs used by UDAs (CASSANDRA-10721)
 + * Add query time validation method on Index (CASSANDRA-11043)
 + * Avoid potential AssertionError in mixed version cluster (CASSANDRA-11128)
 + * Properly handle hinted handoff after topology changes (CASSANDRA-5902)
 + * AssertionError when listing sstable files on inconsistent disk state (CASSANDRA-11156)
 + * Fix wrong rack counting and invalid conditions check for TokenAllocation
 +   (CASSANDRA-11139)
 + * Avoid creating empty hint files (CASSANDRA-11090)
 + * Fix leak detection strong reference loop using weak reference (CASSANDRA-11120)
 + * Configurie BatchlogManager to stop delayed tasks on shutdown (CASSANDRA-11062)
 + * Hadoop integration is incompatible with Cassandra Driver 3.0.0 (CASSANDRA-11001)
 + * Add dropped_columns to the list of schema table so it gets handled
 +   properly (CASSANDRA-11050)
 + * Fix NPE when using forceRepairRangeAsync without DC (CASSANDRA-11239)
 +Merged from 2.2:
 + * Unresolved hostname leads to replace being ignored (CASSANDRA-11210)
   * Range.compareTo() violates the contract of Comparable (CASSANDRA-11216)
   * Avoid NPE when serializing ErrorMessage with null message (CASSANDRA-11167)
   * Replacing an aggregate with a new version doesn't reset INITCOND (CASSANDRA-10840)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/49c616cf/bin/cqlsh.py
----------------------------------------------------------------------
diff --cc bin/cqlsh.py
index 9dd9634,9082d72..83dbeed
--- a/bin/cqlsh.py
+++ b/bin/cqlsh.py
@@@ -602,24 -593,25 +602,44 @@@ def insert_driver_hooks()
  
  
  def extend_cql_deserialization():
-     # The python driver returns BLOBs as string, but we expect them as bytearrays
+     """
 -    The python driver returns BLOBs as string, but we expect them as bytearrays; therefore we change
++    The python driver returns BLOBs as string, but we expect them as bytearrays
+     the implementation of cassandra.cqltypes.BytesType.deserialize.
+ 
+     The deserializers package exists only when the driver has been compiled with cython extensions and
+     cassandra.deserializers.DesBytesType replaces cassandra.cqltypes.BytesType.deserialize.
+ 
+     DesBytesTypeByteArray is a fast deserializer that converts blobs into bytearrays but it was
+     only introduced recently (3.1.0). If it is available we use it, otherwise we remove
+     cassandra.deserializers.DesBytesType so that we fall back onto cassandra.cqltypes.BytesType.deserialize
+     just like in the case where no cython extensions are present.
+     """
+     if hasattr(cassandra, 'deserializers'):
+         if hasattr(cassandra.deserializers, 'DesBytesTypeByteArray'):
+             cassandra.deserializers.DesBytesType = cassandra.deserializers.DesBytesTypeByteArray
+         else:
+             del cassandra.deserializers.DesBytesType
+ 
      cassandra.cqltypes.BytesType.deserialize = staticmethod(lambda byts, protocol_version: bytearray(byts))
 +
 +    class DateOverFlowWarning(RuntimeWarning):
 +        pass
 +
 +    # Native datetime types blow up outside of datetime.[MIN|MAX]_YEAR. We will fall back to an int timestamp
 +    def deserialize_date_fallback_int(byts, protocol_version):
 +        timestamp_ms = int64_unpack(byts)
 +        try:
 +            return datetime_from_timestamp(timestamp_ms / 1000.0)
 +        except OverflowError:
 +            warnings.warn(DateOverFlowWarning("Some timestamps are larger than Python datetime can represent. Timestamps are displayed in milliseconds from epoch."))
 +            return timestamp_ms
 +
 +    cassandra.cqltypes.DateType.deserialize = staticmethod(deserialize_date_fallback_int)
 +
++    if hasattr(cassandra, 'deserializers'):
++        del cassandra.deserializers.DesDateType
++
 +    # Return cassandra.cqltypes.EMPTY instead of None for empty values
      cassandra.cqltypes.CassandraType.support_empty_values = True
  
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/49c616cf/pylib/cqlshlib/copyutil.py
----------------------------------------------------------------------
diff --cc pylib/cqlshlib/copyutil.py
index aeb2d0b,2755dd5..6be990d
--- a/pylib/cqlshlib/copyutil.py
+++ b/pylib/cqlshlib/copyutil.py
@@@ -1484,7 -1714,7 +1714,13 @@@ class ImportConversion(object)
  
              m = p.match(val)
              if not m:
--                raise ValueError("can't interpret %r as a date with this format: %s" % (val, self.date_time_format))
++                try:
++                    # in case of overflow COPY TO prints dates as milliseconds from the epoch, see
++                    # deserialize_date_fallback_int in cqlsh.py
++                    return int(val)
++                except ValueError:
++                    raise ValueError("can't interpret %r as a date with format %s or as int" % (val,
++                                                                                                self.date_time_format))
  
              # https://docs.python.org/2/library/time.html#time.struct_time
              tval = time.struct_time((int(m.group(1)), int(m.group(2)), int(m.group(3)),  # year, month, day


[09/23] cassandra git commit: Merge commit 'c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc' into cassandra-2.2

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b74ffeaf/pylib/cqlshlib/copyutil.py
----------------------------------------------------------------------
diff --cc pylib/cqlshlib/copyutil.py
index aeb2d0b,cd03765..2755dd5
--- a/pylib/cqlshlib/copyutil.py
+++ b/pylib/cqlshlib/copyutil.py
@@@ -45,9 -51,13 +51,13 @@@ from cassandra.util import Date, Tim
  
  from cql3handling import CqlRuleSet
  from displaying import NO_COLOR_MAP
 -from formatting import format_value_default, EMPTY, get_formatter
 +from formatting import format_value_default, DateTimeFormat, EMPTY, get_formatter
  from sslhandling import ssl_settings
  
+ PROFILE_ON = False
+ STRACE_ON = False
+ IS_LINUX = platform.system() == 'Linux'
+ 
  CopyOptions = namedtuple('CopyOptions', 'copy dialect unrecognized')
  
  
@@@ -164,13 -252,12 +252,13 @@@ class CopyTask(object)
          # in the page size or 10 seconds if pagesize is smaller
          copy_options['pagetimeout'] = int(opts.pop('pagetimeout', max(10, 10 * (copy_options['pagesize'] / 1000))))
          copy_options['maxattempts'] = int(opts.pop('maxattempts', 5))
 -        copy_options['dtformats'] = opts.pop('datetimeformat', shell.display_time_format)
 +        copy_options['dtformats'] = DateTimeFormat(opts.pop('datetimeformat', shell.display_timestamp_format),
 +                                                   shell.display_date_format, shell.display_nanotime_format)
          copy_options['float_precision'] = shell.display_float_precision
-         copy_options['chunksize'] = int(opts.pop('chunksize', 1000))
+         copy_options['chunksize'] = int(opts.pop('chunksize', 5000))
          copy_options['ingestrate'] = int(opts.pop('ingestrate', 100000))
          copy_options['maxbatchsize'] = int(opts.pop('maxbatchsize', 20))
-         copy_options['minbatchsize'] = int(opts.pop('minbatchsize', 2))
+         copy_options['minbatchsize'] = int(opts.pop('minbatchsize', 10))
          copy_options['reportfrequency'] = float(opts.pop('reportfrequency', 0.25))
          copy_options['consistencylevel'] = shell.consistency_level
          copy_options['decimalsep'] = opts.pop('decimalsep', '.')
@@@ -1392,9 -1593,37 +1594,37 @@@ class ImportConversion(object)
          self.primary_key_indexes = [self.columns.index(col.name) for col in self.table_meta.primary_key]
          self.partition_key_indexes = [self.columns.index(col.name) for col in self.table_meta.partition_key]
  
+         if statement is None:
+             self.use_prepared_statements = False
+             statement = self._get_primary_key_statement(parent, table_meta)
+         else:
+             self.use_prepared_statements = True
+ 
          self.proto_version = statement.protocol_version
-         self.cqltypes = dict([(c.name, c.type) for c in statement.column_metadata])
-         self.converters = dict([(c.name, self._get_converter(c.type)) for c in statement.column_metadata])
+ 
+         # the cql types and converters for the prepared statement, either the full statement or only the primary keys
+         self.cqltypes = [c.type for c in statement.column_metadata]
+         self.converters = [self._get_converter(c.type) for c in statement.column_metadata]
+ 
+         # the cql types for the entire statement, these are the same as the types above but
+         # only when using prepared statements
 -        self.coltypes = [table_meta.columns[name].typestring for name in parent.valid_columns]
++        self.coltypes = [table_meta.columns[name].cql_type for name in parent.valid_columns]
+         # these functions are used for non-prepared statements to protect values with quotes if required
+         self.protectors = [protect_value if t in ('ascii', 'text', 'timestamp', 'date', 'time', 'inet') else lambda v: v
+                            for t in self.coltypes]
+ 
+     @staticmethod
+     def _get_primary_key_statement(parent, table_meta):
+         """
+         We prepare a query statement to find out the types of the partition key columns so we can
+         route the update query to the correct replicas. As far as I understood this is the easiest
+         way to find out the types of the partition columns, we will never use this prepared statement
+         """
+         where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
+         select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(parent.ks),
+                                                          protect_name(parent.table),
+                                                          where_clause)
+         return parent.session.prepare(select_query)
  
      def _get_converter(self, cql_type):
          """
@@@ -1695,67 -2007,88 +2008,88 @@@ class ImportProcess(ChildProcess)
              self._session.cluster.shutdown()
          ChildProcess.close(self)
  
-     def run_counter(self, table_meta):
-         """
-         Main run method for tables that contain counter columns.
-         """
-         query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
- 
-         # We prepare a query statement to find out the types of the partition key columns so we can
-         # route the update query to the correct replicas. As far as I understood this is the easiest
-         # way to find out the types of the partition columns, we will never use this prepared statement
-         where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
-         select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(self.ks), protect_name(self.table), where_clause)
-         conv = ImportConversion(self, table_meta, self.session.prepare(select_query))
- 
-         while True:
-             batch = self.inmsg.get()
-             try:
-                 for b in self.split_batches(batch, conv):
-                     self.send_counter_batch(query, conv, b)
+     def make_params(self):
+         metadata = self.session.cluster.metadata
+         table_meta = metadata.keyspaces[self.ks].tables[self.table]
+ 
+         prepared_statement = None
 -        is_counter = ("counter" in [table_meta.columns[name].typestring for name in self.valid_columns])
++        is_counter = ("counter" in [table_meta.columns[name].cql_type for name in self.valid_columns])
+         if is_counter:
+             query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
+             make_statement = self.wrap_make_statement(self.make_counter_batch_statement)
+         elif self.use_prepared_statements:
+             query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
+                                                             protect_name(self.table),
+                                                             ', '.join(protect_names(self.valid_columns),),
+                                                             ', '.join(['?' for _ in self.valid_columns]))
+ 
+             query = self.session.prepare(query)
+             query.consistency_level = self.consistency_level
+             prepared_statement = query
+             make_statement = self.wrap_make_statement(self.make_prepared_batch_statement)
+         else:
+             query = 'INSERT INTO %s.%s (%s) VALUES (%%s)' % (protect_name(self.ks),
+                                                              protect_name(self.table),
+                                                              ', '.join(protect_names(self.valid_columns),))
+             make_statement = self.wrap_make_statement(self.make_non_prepared_batch_statement)
  
-             except Exception, exc:
-                 self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                 if self.debug:
-                     traceback.print_exc(exc)
+         conv = ImportConversion(self, table_meta, prepared_statement)
+         tm = TokenMap(self.ks, self.hostname, self.local_dc, self.session)
+         return query, conv, tm, make_statement
  
-     def run_normal(self, table_meta):
+     def inner_run(self, query, conv, tm, make_statement):
          """
-         Main run method for normal tables, i.e. tables that do not contain counter columns.
+         Main run method. Note that we bind self methods that are called inside loops
+         for performance reasons.
          """
-         query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
-                                                         protect_name(self.table),
-                                                         ', '.join(protect_names(self.valid_columns),),
-                                                         ', '.join(['?' for _ in self.valid_columns]))
+         self.query = query
+         self.conv = conv
+         self.make_statement = make_statement
  
-         query_statement = self.session.prepare(query)
-         query_statement.consistency_level = self.consistency_level
-         conv = ImportConversion(self, table_meta, query_statement)
+         convert_rows = self.convert_rows
+         split_into_batches = self.split_into_batches
+         result_callback = self.result_callback
+         err_callback = self.err_callback
+         session = self.session
  
          while True:
-             batch = self.inmsg.get()
+             chunk = self.inmsg.recv()
+             if chunk is None:
+                 break
+ 
              try:
-                 for b in self.split_batches(batch, conv):
-                     self.send_normal_batch(conv, query_statement, b)
+                 chunk['rows'] = convert_rows(conv, chunk)
+                 for replicas, batch in split_into_batches(chunk, conv, tm):
+                     statement = make_statement(query, conv, chunk, batch, replicas)
+                     future = session.execute_async(statement)
+                     future.add_callbacks(callback=result_callback, callback_args=(batch, chunk),
+                                          errback=err_callback, errback_args=(batch, chunk, replicas))
  
              except Exception, exc:
-                 self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                 if self.debug:
-                     traceback.print_exc(exc)
+                 self.report_error(exc, chunk, chunk['rows'])
  
-     def send_counter_batch(self, query_text, conv, batch):
-         if self.test_failures and self.maybe_inject_failures(batch):
-             return
+     def wrap_make_statement(self, inner_make_statement):
+         def make_statement(query, conv, chunk, batch, replicas):
+             try:
+                 return inner_make_statement(query, conv, batch, replicas)
+             except Exception, exc:
+                 print "Failed to make batch statement: {}".format(exc)
+                 self.report_error(exc, chunk, batch['rows'])
+                 return None
  
-         error_rows = []
-         batch_statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+         def make_statement_with_failures(query, conv, chunk, batch, replicas):
+             failed_batch = self.maybe_inject_failures(batch)
+             if failed_batch:
+                 return failed_batch
+             return make_statement(query, conv, chunk, batch, replicas)
  
-         for r in batch['rows']:
-             row = self.filter_row_values(r)
-             if len(row) != len(self.valid_columns):
-                 error_rows.append(row)
-                 continue
+         return make_statement_with_failures if self.test_failures else make_statement
  
+     def make_counter_batch_statement(self, query, conv, batch, replicas):
+         statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+         statement.replicas = replicas
+         statement.keyspace = self.ks
+         for row in batch['rows']:
              where_clause = []
              set_clause = []
              for i, value in enumerate(row):


[02/23] cassandra git commit: COPY FROM on large datasets: fix progress report and debug performance

Posted by sl...@apache.org.
COPY FROM on large datasets: fix progress report and debug performance

patch by Stefania Alborghetti; reviewed by Adam Holmberg for CASSANDRA-11053


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c3d2f26f
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c3d2f26f
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c3d2f26f

Branch: refs/heads/cassandra-2.2
Commit: c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc
Parents: 0129f70
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Thu Jan 28 14:31:55 2016 +0800
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Mar 8 10:19:13 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                |    1 +
 bin/cqlsh                  |   28 +-
 pylib/cqlshlib/copyutil.py | 1160 +++++++++++++++++++++++++--------------
 pylib/cqlshlib/util.py     |   35 +-
 pylib/setup.py             |    2 +
 5 files changed, 809 insertions(+), 417 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index eed9035..d6b085c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.14
+ * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
  * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
  * Don't remove FailureDetector history on removeEndpoint (CASSANDRA-10371)
  * Only notify if repair status changed (CASSANDRA-11172)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/bin/cqlsh
----------------------------------------------------------------------
diff --git a/bin/cqlsh b/bin/cqlsh
index 7a39636..374e588 100755
--- a/bin/cqlsh
+++ b/bin/cqlsh
@@ -417,7 +417,7 @@ COPY_COMMON_OPTIONS = ['DELIMITER', 'QUOTE', 'ESCAPE', 'HEADER', 'NULL', 'DATETI
                        'MAXATTEMPTS', 'REPORTFREQUENCY', 'DECIMALSEP', 'THOUSANDSSEP', 'BOOLSTYLE',
                        'NUMPROCESSES', 'CONFIGFILE', 'RATEFILE']
 COPY_FROM_OPTIONS = ['CHUNKSIZE', 'INGESTRATE', 'MAXBATCHSIZE', 'MINBATCHSIZE', 'MAXROWS',
-                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE']
+                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE', 'PREPAREDSTATEMENTS']
 COPY_TO_OPTIONS = ['ENCODING', 'PAGESIZE', 'PAGETIMEOUT', 'BEGINTOKEN', 'ENDTOKEN', 'MAXOUTPUTSIZE', 'MAXREQUESTS']
 
 
@@ -533,8 +533,23 @@ def insert_driver_hooks():
 
 def extend_cql_deserialization():
     """
-    The python driver returns BLOBs as string, but we expect them as bytearrays
+    The python driver returns BLOBs as string, but we expect them as bytearrays; therefore we change
+    the implementation of cassandra.cqltypes.BytesType.deserialize.
+
+    The deserializers package exists only when the driver has been compiled with cython extensions and
+    cassandra.deserializers.DesBytesType replaces cassandra.cqltypes.BytesType.deserialize.
+
+    DesBytesTypeByteArray is a fast deserializer that converts blobs into bytearrays but it was
+    only introduced recently (3.1.0). If it is available we use it, otherwise we remove
+    cassandra.deserializers.DesBytesType so that we fall back onto cassandra.cqltypes.BytesType.deserialize
+    just like in the case where no cython extensions are present.
     """
+    if hasattr(cassandra, 'deserializers'):
+        if hasattr(cassandra.deserializers, 'DesBytesTypeByteArray'):
+            cassandra.deserializers.DesBytesType = cassandra.deserializers.DesBytesTypeByteArray
+        else:
+            del cassandra.deserializers.DesBytesType
+
     cassandra.cqltypes.BytesType.deserialize = staticmethod(lambda byts, protocol_version: bytearray(byts))
     cassandra.cqltypes.CassandraType.support_empty_values = True
 
@@ -1534,9 +1549,9 @@ class Shell(cmd.Cmd):
 
         Available COPY FROM options and defaults:
 
-          CHUNKSIZE=1000          - the size of chunks passed to worker processes
+          CHUNKSIZE=5000          - the size of chunks passed to worker processes
           INGESTRATE=100000       - an approximate ingest rate in rows per second
-          MINBATCHSIZE=2          - the minimum size of an import batch
+          MINBATCHSIZE=10         - the minimum size of an import batch
           MAXBATCHSIZE=20         - the maximum size of an import batch
           MAXROWS=-1              - the maximum number of rows, -1 means no maximum
           SKIPROWS=0              - the number of rows to skip
@@ -1545,6 +1560,11 @@ class Shell(cmd.Cmd):
           MAXINSERTERRORS=-1      - the maximum global number of insert errors, -1 means no maximum
           ERRFILE=''              - a file where to store all rows that could not be imported, by default this is
                                     import_ks_table.err where <ks> is your keyspace and <table> is your table name.
+          PREPAREDSTATEMENTS=True - whether to use prepared statements when importing, by default True. Set this to
+                                    False if you don't mind shifting data parsing to the cluster. The cluster will also
+                                    have to compile every batch statement. For large and oversized clusters
+                                    this will result in a faster import but for smaller clusters it may generate
+                                    timeouts.
 
         Available COPY TO options and defaults:
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/pylib/cqlshlib/copyutil.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/copyutil.py b/pylib/cqlshlib/copyutil.py
index f9e4a85..cd03765 100644
--- a/pylib/cqlshlib/copyutil.py
+++ b/pylib/cqlshlib/copyutil.py
@@ -1,3 +1,5 @@
+# cython: profile=True
+
 # Licensed to the Apache Software Foundation (ASF) under one
 # or more contributor license agreements.  See the NOTICE file
 # distributed with this work for additional information
@@ -21,25 +23,29 @@ import json
 import glob
 import multiprocessing as mp
 import os
-import Queue
+import platform
+import random
 import re
 import struct
 import sys
 import time
 import traceback
 
+from bisect import bisect_right
 from calendar import timegm
-from collections import defaultdict, deque, namedtuple
+from collections import defaultdict, namedtuple
 from decimal import Decimal
 from random import randrange
 from StringIO import StringIO
+from select import select
 from threading import Lock
 from uuid import UUID
+from util import profile_on, profile_off
 
 from cassandra.cluster import Cluster
 from cassandra.cqltypes import ReversedType, UserType
-from cassandra.metadata import protect_name, protect_names
-from cassandra.policies import RetryPolicy, WhiteListRoundRobinPolicy, TokenAwarePolicy, DCAwareRoundRobinPolicy
+from cassandra.metadata import protect_name, protect_names, protect_value
+from cassandra.policies import RetryPolicy, WhiteListRoundRobinPolicy, DCAwareRoundRobinPolicy
 from cassandra.query import BatchStatement, BatchType, SimpleStatement, tuple_factory
 from cassandra.util import Date, Time
 
@@ -48,6 +54,10 @@ from displaying import NO_COLOR_MAP
 from formatting import format_value_default, EMPTY, get_formatter
 from sslhandling import ssl_settings
 
+PROFILE_ON = False
+STRACE_ON = False
+IS_LINUX = platform.system() == 'Linux'
+
 CopyOptions = namedtuple('CopyOptions', 'copy dialect unrecognized')
 
 
@@ -59,6 +69,81 @@ def safe_normpath(fname):
     return os.path.normpath(os.path.expanduser(fname)) if fname else fname
 
 
+class OneWayChannel(object):
+    """
+    A one way pipe protected by two process level locks, one for reading and one for writing.
+    """
+    def __init__(self):
+        self.reader, self.writer = mp.Pipe(duplex=False)
+        self.rlock = mp.Lock()
+        self.wlock = mp.Lock()
+
+    def send(self, obj):
+        with self.wlock:
+            self.writer.send(obj)
+
+    def recv(self):
+        with self.rlock:
+            return self.reader.recv()
+
+    def close(self):
+        self.reader.close()
+        self.writer.close()
+
+
+class OneWayChannels(object):
+    """
+    A group of one way channels.
+    """
+    def __init__(self, num_channels):
+        self.channels = [OneWayChannel() for _ in xrange(num_channels)]
+        self._readers = [ch.reader for ch in self.channels]
+        self._rlocks = [ch.rlock for ch in self.channels]
+        self._rlocks_by_readers = dict([(ch.reader, ch.rlock) for ch in self.channels])
+        self.num_channels = num_channels
+
+        self.recv = self.recv_select if IS_LINUX else self.recv_polling
+
+    def recv_select(self, timeout):
+        """
+        Implementation of the recv method for Linux, where select is available. Receive an object from
+        all pipes that are ready for reading without blocking.
+        """
+        readable, _, _ = select(self._readers, [], [], timeout)
+        for r in readable:
+            with self._rlocks_by_readers[r]:
+                try:
+                    yield r.recv()
+                except EOFError:
+                    continue
+
+    def recv_polling(self, timeout):
+        """
+        Implementation of the recv method for platforms where select() is not available for pipes.
+        We poll on all of the readers with a very small timeout. We stop when the timeout specified
+        has been received but we may exceed it since we check all processes during each sweep.
+        """
+        start = time.time()
+        while True:
+            for i, r in enumerate(self._readers):
+                with self._rlocks[i]:
+                    if r.poll(0.000000001):
+                        try:
+                            yield r.recv()
+                        except EOFError:
+                            continue
+
+            if time.time() - start > timeout:
+                break
+
+    def close(self):
+        for ch in self.channels:
+            try:
+                ch.close()
+            except:
+                pass
+
+
 class CopyTask(object):
     """
     A base class for ImportTask and ExportTask
@@ -72,15 +157,18 @@ class CopyTask(object):
         self.protocol_version = protocol_version
         self.config_file = config_file
         # do not display messages when exporting to STDOUT
-        self.printmsg = self._printmsg if self.fname is not None or direction == 'in' else lambda _, eol='\n': None
+        self.printmsg = self._printmsg if self.fname is not None or direction == 'from' else lambda _, eol='\n': None
         self.options = self.parse_options(opts, direction)
 
         self.num_processes = self.options.copy['numprocesses']
+        if direction == 'in':
+            self.num_processes += 1  # add the feeder process
+
         self.printmsg('Using %d child processes' % (self.num_processes,))
 
         self.processes = []
-        self.inmsg = mp.Queue()
-        self.outmsg = mp.Queue()
+        self.inmsg = OneWayChannels(self.num_processes)
+        self.outmsg = OneWayChannels(self.num_processes)
 
         self.columns = CopyTask.get_columns(shell, ks, table, columns)
         self.time_start = time.time()
@@ -166,10 +254,10 @@ class CopyTask(object):
         copy_options['maxattempts'] = int(opts.pop('maxattempts', 5))
         copy_options['dtformats'] = opts.pop('datetimeformat', shell.display_time_format)
         copy_options['float_precision'] = shell.display_float_precision
-        copy_options['chunksize'] = int(opts.pop('chunksize', 1000))
+        copy_options['chunksize'] = int(opts.pop('chunksize', 5000))
         copy_options['ingestrate'] = int(opts.pop('ingestrate', 100000))
         copy_options['maxbatchsize'] = int(opts.pop('maxbatchsize', 20))
-        copy_options['minbatchsize'] = int(opts.pop('minbatchsize', 2))
+        copy_options['minbatchsize'] = int(opts.pop('minbatchsize', 10))
         copy_options['reportfrequency'] = float(opts.pop('reportfrequency', 0.25))
         copy_options['consistencylevel'] = shell.consistency_level
         copy_options['decimalsep'] = opts.pop('decimalsep', '.')
@@ -186,6 +274,7 @@ class CopyTask(object):
         copy_options['errfile'] = safe_normpath(opts.pop('errfile', 'import_%s_%s.err' % (self.ks, self.table,)))
         copy_options['ratefile'] = safe_normpath(opts.pop('ratefile', ''))
         copy_options['maxoutputsize'] = int(opts.pop('maxoutputsize', '-1'))
+        copy_options['preparedstatements'] = bool(opts.pop('preparedstatements', 'true').lower() == 'true')
 
         self.check_options(copy_options)
         return CopyOptions(copy=copy_options, dialect=dialect_options, unrecognized=opts)
@@ -206,14 +295,17 @@ class CopyTask(object):
     def get_num_processes(cap):
         """
         Pick a reasonable number of child processes. We need to leave at
-        least one core for the parent process.  This doesn't necessarily
-        need to be capped, but 4 is currently enough to keep
-        a single local Cassandra node busy so we use this for import, whilst
-        for export we use 16 since we can connect to multiple Cassandra nodes.
-        Eventually this parameter will become an option.
+        least one core for the parent process.
+        """
+        return max(1, min(cap, CopyTask.get_num_cores() - 1))
+
+    @staticmethod
+    def get_num_cores():
+        """
+        Return the number of cores if available.
         """
         try:
-            return max(1, min(cap, mp.cpu_count() - 1))
+            return mp.cpu_count()
         except NotImplementedError:
             return 1
 
@@ -244,28 +336,40 @@ class CopyTask(object):
         return shell.get_column_names(ks, table) if not columns else columns
 
     def close(self):
-        for process in self.processes:
-            process.terminate()
-
+        self.stop_processes()
         self.inmsg.close()
         self.outmsg.close()
 
     def num_live_processes(self):
         return sum(1 for p in self.processes if p.is_alive())
 
+    @staticmethod
+    def get_pid():
+        return os.getpid() if hasattr(os, 'getpid') else None
+
+    @staticmethod
+    def trace_process(pid):
+        if pid and STRACE_ON:
+            os.system("strace -vvvv -c -o strace.{pid}.out -e trace=all -p {pid}&".format(pid=pid))
+
+    def start_processes(self):
+        for i, process in enumerate(self.processes):
+            process.start()
+            self.trace_process(process.pid)
+
+        self.trace_process(self.get_pid())
+
+    def stop_processes(self):
+        for process in self.processes:
+            process.terminate()
+
     def make_params(self):
         """
         Return a dictionary of parameters to be used by the worker processes.
         On Windows this dictionary must be pickle-able.
-
-        inmsg is the message queue flowing from parent to child process, so outmsg from the parent point
-        of view and, vice-versa,  outmsg is the message queue flowing from child to parent, so inmsg
-        from the parent point of view, hence the two are swapped below.
         """
         shell = self.shell
-        return dict(inmsg=self.outmsg,  # see comment above
-                    outmsg=self.inmsg,  # see comment above
-                    ks=self.ks,
+        return dict(ks=self.ks,
                     table=self.table,
                     local_dc=self.local_dc,
                     columns=self.columns,
@@ -281,6 +385,17 @@ class CopyTask(object):
                     debug=shell.debug
                     )
 
+    def update_params(self, params, i):
+        """
+        Add the communication channels to the parameters to be passed to the worker process:
+            inmsg is the message queue flowing from parent to child process, so outmsg from the parent point
+            of view and, vice-versa,  outmsg is the message queue flowing from child to parent, so inmsg
+            from the parent point of view, hence the two are swapped below.
+        """
+        params['inmsg'] = self.outmsg.channels[i]
+        params['outmsg'] = self.inmsg.channels[i]
+        return params
+
 
 class ExportWriter(object):
     """
@@ -414,10 +529,9 @@ class ExportTask(CopyTask):
 
         params = self.make_params()
         for i in xrange(self.num_processes):
-            self.processes.append(ExportProcess(params))
+            self.processes.append(ExportProcess(self.update_params(params, i)))
 
-        for process in self.processes:
-            process.start()
+        self.start_processes()
 
         try:
             self.export_records(ranges)
@@ -468,11 +582,12 @@ class ExportTask(CopyTask):
 
             return ret
 
-        def make_range_data(replicas=[]):
+        def make_range_data(replicas=None):
             hosts = []
-            for r in replicas:
-                if r.is_up and r.datacenter == local_dc:
-                    hosts.append(r.address)
+            if replicas:
+                for r in replicas:
+                    if r.is_up and r.datacenter == local_dc:
+                        hosts.append(r.address)
             if not hosts:
                 hosts.append(hostname)  # fallback to default host if no replicas in current dc
             return {'hosts': tuple(hosts), 'attempts': 0, 'rows': 0}
@@ -542,10 +657,13 @@ class ExportTask(CopyTask):
             return None
 
     def send_work(self, ranges, tokens_to_send):
+        i = 0
         for token_range in tokens_to_send:
-            self.outmsg.put((token_range, ranges[token_range]))
+            self.outmsg.channels[i].send((token_range, ranges[token_range]))
             ranges[token_range]['attempts'] += 1
 
+            i = i + 1 if i < self.num_processes - 1 else 0
+
     def export_records(self, ranges):
         """
         Send records to child processes and monitor them by collecting their results
@@ -568,8 +686,7 @@ class ExportTask(CopyTask):
         succeeded = 0
         failed = 0
         while (failed + succeeded) < total_requests and self.num_live_processes() == num_processes:
-            try:
-                token_range, result = self.inmsg.get(timeout=1.0)
+            for token_range, result in self.inmsg.recv(timeout=0.1):
                 if token_range is None and result is None:  # a request has finished
                     succeeded += 1
                 elif isinstance(result, Exception):  # an error occurred
@@ -594,8 +711,6 @@ class ExportTask(CopyTask):
                     self.writer.write(data, num)
                     meter.increment(n=num)
                     ranges[token_range]['rows'] += num
-            except Queue.Empty:
-                pass
 
         if self.num_live_processes() < len(processes):
             for process in processes:
@@ -612,7 +727,7 @@ class ExportTask(CopyTask):
                        self.describe_interval(time.time() - self.time_start)))
 
 
-class ImportReader(object):
+class FilesReader(object):
     """
     A wrapper around a csv reader to keep track of when we have
     exhausted reading input files. We are passed a comma separated
@@ -620,18 +735,15 @@ class ImportReader(object):
     We generate a source generator and we read each source one
     by one.
     """
-    def __init__(self, task):
-        self.shell = task.shell
-        self.options = task.options
-        self.printmsg = task.printmsg
-        self.chunk_size = self.options.copy['chunksize']
-        self.header = self.options.copy['header']
-        self.max_rows = self.options.copy['maxrows']
-        self.skip_rows = self.options.copy['skiprows']
-        self.sources = self.get_source(task.fname)
+    def __init__(self, fname, options):
+        self.chunk_size = options.copy['chunksize']
+        self.header = options.copy['header']
+        self.max_rows = options.copy['maxrows']
+        self.skip_rows = options.copy['skiprows']
+        self.fname = fname
+        self.sources = None  # must be created later due to pickle problems on Windows
         self.num_sources = 0
         self.current_source = None
-        self.current_reader = None
         self.num_read = 0
 
     def get_source(self, paths):
@@ -640,35 +752,33 @@ class ImportReader(object):
          wrapping the source input, file name and a boolean indicating
          if it requires closing.
         """
-        shell = self.shell
-        LineSource = namedtuple('LineSource', 'input close fname')
-
         def make_source(fname):
             try:
-                ret = LineSource(input=open(fname, 'rb'), close=True, fname=fname)
-                return ret
+                return open(fname, 'rb')
             except IOError, e:
-                shell.printerr("Can't open %r for reading: %s" % (fname, e))
+                self.printmsg("Can't open %r for reading: %s" % (fname, e))
                 return None
 
-        if paths is None:
-            self.printmsg("[Use \. on a line by itself to end input]")
-            yield LineSource(input=shell.use_stdin_reader(prompt='[copy] ', until=r'\.'), close=False, fname='')
-        else:
-            for path in paths.split(','):
-                path = path.strip()
-                if os.path.isfile(path):
-                    yield make_source(path)
-                else:
-                    for f in glob.glob(path):
-                        yield (make_source(f))
+        for path in paths.split(','):
+            path = path.strip()
+            if os.path.isfile(path):
+                yield make_source(path)
+            else:
+                for f in glob.glob(path):
+                    yield (make_source(f))
+
+    @staticmethod
+    def printmsg(msg, eol='\n'):
+        sys.stdout.write(msg + eol)
+        sys.stdout.flush()
 
     def start(self):
+        self.sources = self.get_source(self.fname)
         self.next_source()
 
     @property
     def exhausted(self):
-        return not self.current_reader
+        return not self.current_source
 
     def next_source(self):
         """
@@ -679,40 +789,34 @@ class ImportReader(object):
         while self.current_source is None:
             try:
                 self.current_source = self.sources.next()
-                if self.current_source and self.current_source.fname:
+                if self.current_source:
                     self.num_sources += 1
             except StopIteration:
                 return False
 
         if self.header:
-            self.current_source.input.next()
+            self.current_source.next()
 
-        self.current_reader = csv.reader(self.current_source.input, **self.options.dialect)
         return True
 
     def close_current_source(self):
         if not self.current_source:
             return
 
-        if self.current_source.close:
-            self.current_source.input.close()
-        elif self.shell.tty:
-            print
-
+        self.current_source.close()
         self.current_source = None
-        self.current_reader = None
 
     def close(self):
         self.close_current_source()
 
     def read_rows(self, max_rows):
-        if not self.current_reader:
+        if not self.current_source:
             return []
 
         rows = []
         for i in xrange(min(max_rows, self.chunk_size)):
             try:
-                row = self.current_reader.next()
+                row = self.current_source.next()
                 self.num_read += 1
 
                 if 0 <= self.max_rows < self.num_read:
@@ -729,13 +833,91 @@ class ImportReader(object):
         return filter(None, rows)
 
 
-class ImportErrors(object):
+class PipeReader(object):
     """
-    A small class for managing import errors
+    A class for reading rows received on a pipe, this is used for reading input from STDIN
+    """
+    def __init__(self, inmsg, options):
+        self.inmsg = inmsg
+        self.chunk_size = options.copy['chunksize']
+        self.header = options.copy['header']
+        self.max_rows = options.copy['maxrows']
+        self.skip_rows = options.copy['skiprows']
+        self.num_read = 0
+        self.exhausted = False
+        self.num_sources = 1
+
+    def start(self):
+        pass
+
+    def read_rows(self, max_rows):
+        rows = []
+        for i in xrange(min(max_rows, self.chunk_size)):
+            row = self.inmsg.recv()
+            if row is None:
+                self.exhausted = True
+                break
+
+            self.num_read += 1
+            if 0 <= self.max_rows < self.num_read:
+                self.exhausted = True
+                break  # max rows exceeded
+
+            if self.header or self.num_read < self.skip_rows:
+                self.header = False  # skip header or initial skip_rows rows
+                continue
+
+            rows.append(row)
+
+        return rows
+
+
+class ImportProcessResult(object):
+    """
+    An object sent from ImportProcess instances to the parent import task in order to indicate progress.
+    """
+    def __init__(self, imported=0):
+        self.imported = imported
+
+
+class FeedingProcessResult(object):
+    """
+    An object sent from FeedingProcess instances to the parent import task in order to indicate progress.
+    """
+    def __init__(self, sent, reader):
+        self.sent = sent
+        self.num_sources = reader.num_sources
+        self.skip_rows = reader.skip_rows
+
+
+class ImportTaskError(object):
+    """
+    An object sent from child processes (feeder or workers) to the parent import task to indicate an error.
+    """
+    def __init__(self, name, msg, rows=None, attempts=1, final=True):
+        self.name = name
+        self.msg = msg
+        self.rows = rows if rows else []
+        self.attempts = attempts
+        self.final = final
+
+    def is_parse_error(self):
+        """
+        We treat read and parse errors as unrecoverable and we have different global counters for giving up when
+        a maximum has been reached. We consider value and type errors as parse errors as well since they
+        are typically non recoverable.
+        """
+        name = self.name
+        return name.startswith('ValueError') or name.startswith('TypeError') or \
+            name.startswith('ParseError') or name.startswith('IndexError') or name.startswith('ReadError')
+
+
+class ImportErrorHandler(object):
+    """
+    A class for managing import errors
     """
     def __init__(self, task):
         self.shell = task.shell
-        self.reader = task.reader
         self.options = task.options
         self.printmsg = task.printmsg
         self.max_attempts = self.options.copy['maxattempts']
@@ -771,42 +953,26 @@ class ImportErrors(object):
             for row in rows:
                 writer.writerow(row)
 
-    def handle_error(self, err, batch):
+    def handle_error(self, err):
         """
         Handle an error by printing the appropriate error message and incrementing the correct counter.
-        Return true if we should retry this batch, false if the error is non-recoverable
         """
         shell = self.shell
-        err = str(err)
 
-        if self.is_parse_error(err):
-            self.parse_errors += len(batch['rows'])
-            self.add_failed_rows(batch['rows'])
-            shell.printerr("Failed to import %d rows: %s -  given up without retries"
-                           % (len(batch['rows']), err))
-            return False
+        if err.is_parse_error():
+            self.parse_errors += len(err.rows)
+            self.add_failed_rows(err.rows)
+            shell.printerr("Failed to import %d rows: %s - %s,  given up without retries"
+                           % (len(err.rows), err.name, err.msg))
         else:
-            self.insert_errors += len(batch['rows'])
-            if batch['attempts'] < self.max_attempts:
-                shell.printerr("Failed to import %d rows: %s -  will retry later, attempt %d of %d"
-                               % (len(batch['rows']), err, batch['attempts'],
-                                  self.max_attempts))
-                return True
+            self.insert_errors += len(err.rows)
+            if not err.final:
+                shell.printerr("Failed to import %d rows: %s - %s,  will retry later, attempt %d of %d"
+                               % (len(err.rows), err.name, err.msg, err.attempts, self.max_attempts))
             else:
-                self.add_failed_rows(batch['rows'])
-                shell.printerr("Failed to import %d rows: %s -  given up after %d attempts"
-                               % (len(batch['rows']), err, batch['attempts']))
-                return False
-
-    @staticmethod
-    def is_parse_error(err):
-        """
-        We treat parse errors as unrecoverable and we have different global counters for giving up when
-        a maximum has been reached. We consider value and type errors as parse errors as well since they
-        are typically non recoverable.
-        """
-        return err.startswith('ValueError') or err.startswith('TypeError') or \
-            err.startswith('ParseError') or err.startswith('IndexError')
+                self.add_failed_rows(err.rows)
+                shell.printerr("Failed to import %d rows: %s - %s,  given up after %d attempts"
+                               % (len(err.rows), err.name, err.msg, err.attempts))
 
 
 class ImportTask(CopyTask):
@@ -818,22 +984,14 @@ class ImportTask(CopyTask):
         CopyTask.__init__(self, shell, ks, table, columns, fname, opts, protocol_version, config_file, 'from')
 
         options = self.options
-        self.ingest_rate = options.copy['ingestrate']
-        self.max_attempts = options.copy['maxattempts']
-        self.header = options.copy['header']
         self.skip_columns = [c.strip() for c in self.options.copy['skipcols'].split(',')]
         self.valid_columns = [c for c in self.columns if c not in self.skip_columns]
         self.table_meta = self.shell.get_table_meta(self.ks, self.table)
-        self.batch_id = 0
         self.receive_meter = RateMeter(log_fcn=self.printmsg,
                                        update_interval=options.copy['reportfrequency'],
                                        log_file=options.copy['ratefile'])
-        self.send_meter = RateMeter(log_fcn=None, update_interval=1)
-        self.reader = ImportReader(self)
-        self.import_errors = ImportErrors(self)
-        self.retries = deque([])
-        self.failed = 0
-        self.succeeded = 0
+        self.error_handler = ImportErrorHandler(self)
+        self.feeding_result = None
         self.sent = 0
 
     def make_params(self):
@@ -861,17 +1019,24 @@ class ImportTask(CopyTask):
         self.printmsg("\nStarting copy of %s.%s with columns %s." % (self.ks, self.table, self.valid_columns))
 
         try:
-            self.reader.start()
             params = self.make_params()
 
-            for i in range(self.num_processes):
-                self.processes.append(ImportProcess(params))
+            for i in range(self.num_processes - 1):
+                self.processes.append(ImportProcess(self.update_params(params, i)))
+
+            feeder = FeedingProcess(self.outmsg.channels[-1], self.inmsg.channels[-1],
+                                    self.outmsg.channels[:-1], self.fname, self.options)
+            self.processes.append(feeder)
 
-            for process in self.processes:
-                process.start()
+            self.start_processes()
+
+            pr = profile_on() if PROFILE_ON else None
 
             self.import_records()
 
+            if pr:
+                profile_off(pr, file_name='parent_profile_%d.txt' % (os.getpid(),))
+
         except Exception, exc:
             shell.printerr(str(exc))
             if shell.debug:
@@ -880,9 +1045,22 @@ class ImportTask(CopyTask):
         finally:
             self.close()
 
-    def close(self):
-        CopyTask.close(self)
-        self.reader.close()
+    def send_stdin_rows(self):
+        """
+        We need to pass stdin rows to the feeder process as it is not safe to pickle or share stdin
+        directly (in case of file the child process would close it). This is a very primitive support
+        for STDIN import in that we we won't start reporting progress until STDIN is fully consumed. I
+        think this is reasonable.
+        """
+        shell = self.shell
+
+        self.printmsg("[Use \. on a line by itself to end input]")
+        for row in shell.use_stdin_reader(prompt='[copy] ', until=r'.'):
+            self.outmsg.channels[-1].send(row)
+
+        self.outmsg.channels[-1].send(None)
+        if shell.tty:
+            print
 
     def import_records(self):
         """
@@ -890,114 +1068,137 @@ class ImportTask(CopyTask):
         Send data (batches or retries) up to the max ingest rate. If we are waiting for stuff to
         receive check the incoming queue.
         """
-        reader = self.reader
-
-        while self.has_more_to_send(reader) or self.has_more_to_receive():
-            if self.has_more_to_send(reader):
-                self.send_batches(reader)
+        if not self.fname:
+            self.send_stdin_rows()
 
-            if self.has_more_to_receive():
-                self.receive()
+        while self.feeding_result is None or self.receive_meter.total_records < self.feeding_result.sent:
+            self.receive_results()
 
-            if self.import_errors.max_exceeded() or not self.all_processes_running():
+            if self.error_handler.max_exceeded() or not self.all_processes_running():
                 break
 
-        if self.import_errors.num_rows_failed:
+        if self.error_handler.num_rows_failed:
             self.shell.printerr("Failed to process %d rows; failed rows written to %s" %
-                                (self.import_errors.num_rows_failed,
-                                 self.import_errors.err_file))
+                                (self.error_handler.num_rows_failed,
+                                 self.error_handler.err_file))
 
         if not self.all_processes_running():
             self.shell.printerr("{} child process(es) died unexpectedly, aborting"
                                 .format(self.num_processes - self.num_live_processes()))
+        else:
+            # it is only safe to write to processes if they are all running because the feeder process
+            # at the moment hangs whilst sending messages to a crashed worker process; in future
+            # we could do something about this by using a BoundedSemaphore to keep track of how many messages are
+            # queued on a pipe
+            for i, _ in enumerate(self.processes):
+                self.outmsg.channels[i].send(None)
+
+            if PROFILE_ON:
+                # allow time for worker processes to write profile results (only works if processes received
+                # the poison pill above)
+                time.sleep(5)
 
         self.printmsg("\n%d rows imported from %d files in %s (%d skipped)." %
                       (self.receive_meter.get_total_records(),
-                       self.reader.num_sources,
+                       self.feeding_result.num_sources if self.feeding_result else 0,
                        self.describe_interval(time.time() - self.time_start),
-                       self.reader.skip_rows))
-
-    def has_more_to_receive(self):
-        return (self.succeeded + self.failed) < self.sent
-
-    def has_more_to_send(self, reader):
-        return (not reader.exhausted) or self.retries
+                       self.feeding_result.skip_rows if self.feeding_result else 0))
 
     def all_processes_running(self):
-        return self.num_live_processes() == self.num_processes
+        return self.num_live_processes() == len(self.processes)
 
-    def receive(self):
-        start_time = time.time()
+    def receive_results(self):
+        """
+        Receive results from the worker processes, which will send the number of rows imported
+        or from the feeder process, which will send the number of rows sent when it has finished sending rows.
+        """
+        aggregate_result = ImportProcessResult()
+        try:
+            for result in self.inmsg.recv(timeout=0.1):
+                if isinstance(result, ImportProcessResult):
+                    aggregate_result.imported += result.imported
+                elif isinstance(result, ImportTaskError):
+                    self.error_handler.handle_error(result)
+                elif isinstance(result, FeedingProcessResult):
+                    self.feeding_result = result
+                else:
+                    raise ValueError("Unexpected result: %s" % (result,))
+        finally:
+            self.receive_meter.increment(aggregate_result.imported)
 
-        while time.time() - start_time < 0.001:
-            try:
-                batch, err = self.inmsg.get(timeout=0.00001)
 
-                if err is None:
-                    self.succeeded += batch['imported']
-                    self.receive_meter.increment(batch['imported'])
-                else:
-                    err = str(err)
+class FeedingProcess(mp.Process):
+    """
+    A process that reads from import sources and sends chunks to worker processes.
+    """
+    def __init__(self, inmsg, outmsg, worker_channels, fname, options):
+        mp.Process.__init__(self, target=self.run)
+        self.inmsg = inmsg
+        self.outmsg = outmsg
+        self.worker_channels = worker_channels
+        self.reader = FilesReader(fname, options) if fname else PipeReader(inmsg, options)
+        self.send_meter = RateMeter(log_fcn=None, update_interval=1)
+        self.ingest_rate = options.copy['ingestrate']
+        self.num_worker_processes = options.copy['numprocesses']
+        self.chunk_id = 0
+
+    def run(self):
+        pr = profile_on() if PROFILE_ON else None
 
-                    if self.import_errors.handle_error(err, batch):
-                        self.retries.append(self.reset_batch(batch))
-                    else:
-                        self.failed += len(batch['rows'])
+        self.inner_run()
 
-            except Queue.Empty:
-                pass
+        if pr:
+            profile_off(pr, file_name='feeder_profile_%d.txt' % (os.getpid(),))
 
-    def send_batches(self, reader):
+    def inner_run(self):
         """
         Send one batch per worker process to the queue unless we have exceeded the ingest rate.
         In the export case we queue everything and let the worker processes throttle using max_requests,
-        here we throttle using the ingest rate in the parent process because of memory usage concerns.
-
-        When we have finished reading the csv file, then send any retries.
+        here we throttle using the ingest rate in the feeding process because of memory usage concerns.
+        When finished we send back to the parent process the total number of rows sent.
         """
-        for _ in xrange(self.num_processes):
-            max_rows = self.ingest_rate - self.send_meter.current_record
-            if max_rows <= 0:
-                self.send_meter.maybe_update()
-                break
+        reader = self.reader
+        reader.start()
+        channels = self.worker_channels
+        sent = 0
+
+        while not reader.exhausted:
+            for ch in channels:
+                try:
+                    max_rows = self.ingest_rate - self.send_meter.current_record
+                    if max_rows <= 0:
+                        self.send_meter.maybe_update(sleep=False)
+                        continue
+
+                    rows = reader.read_rows(max_rows)
+                    if rows:
+                        sent += self.send_chunk(ch, rows)
+                except Exception, exc:
+                    self.outmsg.send(ImportTaskError(exc.__class__.__name__, exc.message))
+
+                if reader.exhausted:
+                    break
 
-            if not reader.exhausted:
-                rows = reader.read_rows(max_rows)
-                if rows:
-                    self.sent += self.send_batch(self.new_batch(rows))
-            elif self.retries:
-                batch = self.retries.popleft()
-                if len(batch['rows']) <= max_rows:
-                    self.send_batch(batch)
-                else:
-                    self.send_batch(self.split_batch(batch, batch['rows'][:max_rows]))
-                    self.retries.append(self.split_batch(batch, batch['rows'][max_rows:]))
-            else:
-                break
+        # send back to the parent process the number of rows sent to the worker processes
+        self.outmsg.send(FeedingProcessResult(sent, reader))
+
+        # wait for poison pill (None)
+        self.inmsg.recv()
 
-    def send_batch(self, batch):
-        batch['attempts'] += 1
-        num_rows = len(batch['rows'])
+    def send_chunk(self, ch, rows):
+        self.chunk_id += 1
+        num_rows = len(rows)
         self.send_meter.increment(num_rows)
-        self.outmsg.put(batch)
+        ch.send({'id': self.chunk_id, 'rows': rows, 'imported': 0, 'num_rows_sent': num_rows})
         return num_rows
 
-    def new_batch(self, rows):
-        self.batch_id += 1
-        return self.make_batch(self.batch_id, rows, 0)
-
-    @staticmethod
-    def reset_batch(batch):
-        batch['imported'] = 0
-        return batch
-
-    @staticmethod
-    def split_batch(batch, rows):
-        return ImportTask.make_batch(batch['id'], rows, batch['attempts'])
+    def close(self):
+        self.reader.close()
+        self.inmsg.close()
+        self.outmsg.close()
 
-    @staticmethod
-    def make_batch(batch_id, rows, attempts):
-        return {'id': batch_id, 'rows': rows, 'attempts': attempts, 'imported': 0}
+        for ch in self.worker_channels:
+            ch.close()
 
 
 class ChildProcess(mp.Process):
@@ -1029,6 +1230,7 @@ class ChildProcess(mp.Process):
         self.decimal_sep = options.copy['decimalsep']
         self.thousands_sep = options.copy['thousandssep']
         self.boolean_styles = options.copy['boolstyle']
+        self.max_attempts = options.copy['maxattempts']
         # Here we inject some failures for testing purposes, only if this environment variable is set
         if os.environ.get('CQLSH_COPY_TEST_FAILURES', ''):
             self.test_failures = json.loads(os.environ.get('CQLSH_COPY_TEST_FAILURES', ''))
@@ -1144,7 +1346,6 @@ class ExportProcess(ChildProcess):
         self.encoding = options.copy['encoding']
         self.float_precision = options.copy['float_precision']
         self.nullval = options.copy['nullval']
-        self.max_attempts = options.copy['maxattempts']
         self.max_requests = options.copy['maxrequests']
 
         self.hosts_to_sessions = dict()
@@ -1172,7 +1373,7 @@ class ExportProcess(ChildProcess):
                 time.sleep(0.001)  # 1 millisecond
                 continue
 
-            token_range, info = self.inmsg.get()
+            token_range, info = self.inmsg.recv()
             self.start_request(token_range, info)
 
     @staticmethod
@@ -1190,7 +1391,7 @@ class ExportProcess(ChildProcess):
     def report_error(self, err, token_range=None):
         msg = self.get_error_message(err, print_traceback=self.debug)
         self.printdebugmsg(msg)
-        self.outmsg.put((token_range, Exception(msg)))
+        self.outmsg.send((token_range, Exception(msg)))
 
     def start_request(self, token_range, info):
         """
@@ -1253,7 +1454,8 @@ class ExportProcess(ChildProcess):
             default_retry_policy=ExpBackoffRetryPolicy(self),
             compression=None,
             control_connection_timeout=self.connect_timeout,
-            connect_timeout=self.connect_timeout)
+            connect_timeout=self.connect_timeout,
+            idle_heartbeat_interval=0)
         session = ExportSession(new_cluster, self)
         self.hosts_to_sessions[host] = session
         return session
@@ -1265,7 +1467,7 @@ class ExportProcess(ChildProcess):
                 self.write_rows_to_csv(token_range, rows)
             else:
                 self.write_rows_to_csv(token_range, rows)
-                self.outmsg.put((None, None))
+                self.outmsg.send((None, None))
                 session.complete_request()
 
         def err_callback(err):
@@ -1286,7 +1488,7 @@ class ExportProcess(ChildProcess):
                 writer.writerow(map(self.format_value, row))
 
             data = (output.getvalue(), len(rows))
-            self.outmsg.put((token_range, data))
+            self.outmsg.send((token_range, data))
             output.close()
 
         except Exception, e:
@@ -1376,7 +1578,7 @@ class ImportConversion(object):
     A class for converting strings to values when importing from csv, used by ImportProcess,
     the parent.
     """
-    def __init__(self, parent, table_meta, statement):
+    def __init__(self, parent, table_meta, statement=None):
         self.ks = parent.ks
         self.table = parent.table
         self.columns = parent.valid_columns
@@ -1391,9 +1593,37 @@ class ImportConversion(object):
         self.primary_key_indexes = [self.columns.index(col.name) for col in self.table_meta.primary_key]
         self.partition_key_indexes = [self.columns.index(col.name) for col in self.table_meta.partition_key]
 
+        if statement is None:
+            self.use_prepared_statements = False
+            statement = self._get_primary_key_statement(parent, table_meta)
+        else:
+            self.use_prepared_statements = True
+
         self.proto_version = statement.protocol_version
-        self.cqltypes = dict([(c.name, c.type) for c in statement.column_metadata])
-        self.converters = dict([(c.name, self._get_converter(c.type)) for c in statement.column_metadata])
+
+        # the cql types and converters for the prepared statement, either the full statement or only the primary keys
+        self.cqltypes = [c.type for c in statement.column_metadata]
+        self.converters = [self._get_converter(c.type) for c in statement.column_metadata]
+
+        # the cql types for the entire statement, these are the same as the types above but
+        # only when using prepared statements
+        self.coltypes = [table_meta.columns[name].typestring for name in parent.valid_columns]
+        # these functions are used for non-prepared statements to protect values with quotes if required
+        self.protectors = [protect_value if t in ('ascii', 'text', 'timestamp', 'date', 'time', 'inet') else lambda v: v
+                           for t in self.coltypes]
+
+    @staticmethod
+    def _get_primary_key_statement(parent, table_meta):
+        """
+        We prepare a query statement to find out the types of the partition key columns so we can
+        route the update query to the correct replicas. As far as I understood this is the easiest
+        way to find out the types of the partition columns, we will never use this prepared statement
+        """
+        where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
+        select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(parent.ks),
+                                                         protect_name(parent.table),
+                                                         where_clause)
+        return parent.session.prepare(select_query)
 
     def _get_converter(self, cql_type):
         """
@@ -1581,27 +1811,25 @@ class ImportConversion(object):
 
         return converters.get(cql_type.typename, convert_unknown)
 
-    def get_row_values(self, row):
+    def convert_row(self, row):
         """
-        Parse the row into a list of row values to be returned
+        Convert the row into a list of parsed values if using prepared statements, else simply apply the
+        protection functions to escape values with quotes when required. Also check on the row length and
+        make sure primary partition key values aren't missing.
         """
-        def convert(n, val):
-            try:
-                return self.converters[self.columns[n]](val)
-            except Exception, e:
-                raise ParseError(e.message)
+        converters = self.converters if self.use_prepared_statements else self.protectors
 
-        ret = [None] * len(row)
-        for i, val in enumerate(row):
-            if val != self.nullval:
-                ret[i] = convert(i, val)
-            else:
-                if i in self.primary_key_indexes:
-                    raise ParseError(self.get_null_primary_key_message(i))
+        if len(row) != len(converters):
+            raise ParseError('Invalid row length %d should be %d' % (len(row), len(converters)))
 
-                ret[i] = None
+        for i in self.primary_key_indexes:
+            if row[i] == self.nullval:
+                raise ParseError(self.get_null_primary_key_message(i))
 
-        return ret
+        try:
+            return [conv(val) for conv, val in zip(converters, row)]
+        except Exception, e:
+            raise ParseError(e.message)
 
     def get_null_primary_key_message(self, idx):
         message = "Cannot insert null value for primary key column '%s'." % (self.columns[idx],)
@@ -1610,31 +1838,111 @@ class ImportConversion(object):
                        " the WITH NULL=<marker> option for COPY."
         return message
 
-    def get_row_partition_key_values(self, row):
+    def get_row_partition_key_values_fcn(self):
         """
-        Return a string composed of the partition key values, serialized and binary packed -
-        as expected by metadata.get_replicas(), see also BoundStatement.routing_key.
+        Return a function to convert a row into a string composed of the partition key values serialized
+        and binary packed (the tokens on the ring). Depending on whether we are using prepared statements, we
+        may have to convert the primary key values first, so we have two different serialize_value implementations.
+        We also return different functions depending on how many partition key indexes we have (single or multiple).
+        See also BoundStatement.routing_key.
         """
-        def serialize(n):
-            try:
-                c, v = self.columns[n], row[n]
-                if v == self.nullval:
-                    raise ParseError(self.get_null_primary_key_message(n))
-                return self.cqltypes[c].serialize(self.converters[c](v), self.proto_version)
-            except Exception, e:
-                raise ParseError(e.message)
+        def serialize_value_prepared(n, v):
+            return self.cqltypes[n].serialize(v, self.proto_version)
+
+        def serialize_value_not_prepared(n, v):
+            return self.cqltypes[n].serialize(self.converters[n](v), self.proto_version)
 
         partition_key_indexes = self.partition_key_indexes
-        if len(partition_key_indexes) == 1:
-            return serialize(partition_key_indexes[0])
-        else:
+        serialize = serialize_value_prepared if self.use_prepared_statements else serialize_value_not_prepared
+
+        def serialize_row_single(row):
+            return serialize(partition_key_indexes[0], row[partition_key_indexes[0]])
+
+        def serialize_row_multiple(row):
             pk_values = []
             for i in partition_key_indexes:
-                val = serialize(i)
+                val = serialize(i, row[i])
                 l = len(val)
                 pk_values.append(struct.pack(">H%dsB" % l, l, val, 0))
             return b"".join(pk_values)
 
+        if len(partition_key_indexes) == 1:
+            return serialize_row_single
+        return serialize_row_multiple
+
+
+class TokenMap(object):
+    """
+    A wrapper around the metadata token map to speed things up by caching ring token *values* and
+    replicas. It is very important that we use the token values, which are primitive types, rather
+    than the tokens classes when calling bisect_right() in split_batches(). If we use primitive values,
+    the bisect is done in compiled code whilst with token classes each comparison requires a call
+    into the interpreter to perform the cmp operation defined in Python. A simple test with 1 million bisect
+    operations on an array of 2048 tokens was done in 0.37 seconds with primitives and 2.25 seconds with
+    token classes. This is significant for large datasets because we need to do a bisect for each single row,
+    and if VNODES are used, the size of the token map can get quite large too.
+    """
+    def __init__(self, ks, hostname, local_dc, session):
+
+        self.ks = ks
+        self.hostname = hostname
+        self.local_dc = local_dc
+        self.metadata = session.cluster.metadata
+
+        self._initialize_ring()
+
+        # Note that refresh metadata is disabled by default and we currenlty do not intercept it
+        # If hosts are added, removed or moved during a COPY operation our token map is no longer optimal
+        # However we can cope with hosts going down and up since we filter for replicas that are up when
+        # making each batch
+
+    def _initialize_ring(self):
+        token_map = self.metadata.token_map
+        if token_map is None:
+            self.ring = [0]
+            self.replicas = [(self.metadata.get_host(self.hostname),)]
+            self.pk_to_token_value = lambda pk: 0
+            return
+
+        token_map.rebuild_keyspace(self.ks, build_if_absent=True)
+        tokens_to_hosts = token_map.tokens_to_hosts_by_ks.get(self.ks, None)
+        from_key = token_map.token_class.from_key
+
+        self.ring = [token.value for token in token_map.ring]
+        self.replicas = [tuple(tokens_to_hosts[token]) for token in token_map.ring]
+        self.pk_to_token_value = lambda pk: from_key(pk).value
+
+    @staticmethod
+    def get_ring_pos(ring, val):
+        idx = bisect_right(ring, val)
+        return idx if idx < len(ring) else 0
+
+    def filter_replicas(self, hosts):
+        shuffled = tuple(sorted(hosts, key=lambda k: random.random()))
+        return filter(lambda r: r.is_up and r.datacenter == self.local_dc, shuffled) if hosts else ()
+
+
+class FastTokenAwarePolicy(DCAwareRoundRobinPolicy):
+    """
+    Send to any replicas attached to the query, or else fall back to DCAwareRoundRobinPolicy
+    """
+
+    def __init__(self, local_dc='', used_hosts_per_remote_dc=0):
+        DCAwareRoundRobinPolicy.__init__(self, local_dc, used_hosts_per_remote_dc)
+
+    def make_query_plan(self, working_keyspace=None, query=None):
+        """
+        Extend TokenAwarePolicy.make_query_plan() so that we choose the same replicas in preference
+        and most importantly we avoid repeating the (slow) bisect
+        """
+        replicas = query.replicas if hasattr(query, 'replicas') else []
+        for r in replicas:
+            yield r
+
+        for r in DCAwareRoundRobinPolicy.make_query_plan(self, working_keyspace, query):
+            if r not in replicas:
+                yield r
+
 
 class ImportProcess(ChildProcess):
 
@@ -1650,7 +1958,12 @@ class ImportProcess(ChildProcess):
         self.max_attempts = options.copy['maxattempts']
         self.min_batch_size = options.copy['minbatchsize']
         self.max_batch_size = options.copy['maxbatchsize']
+        self.use_prepared_statements = options.copy['preparedstatements']
+        self.dialect_options = options.dialect
         self._session = None
+        self.query = None
+        self.conv = None
+        self.make_statement = None
 
     @property
     def session(self):
@@ -1661,12 +1974,13 @@ class ImportProcess(ChildProcess):
                 cql_version=self.cql_version,
                 protocol_version=self.protocol_version,
                 auth_provider=self.auth_provider,
-                load_balancing_policy=TokenAwarePolicy(DCAwareRoundRobinPolicy(local_dc=self.local_dc)),
+                load_balancing_policy=FastTokenAwarePolicy(local_dc=self.local_dc),
                 ssl_options=ssl_settings(self.hostname, self.config_file) if self.ssl else None,
                 default_retry_policy=ExpBackoffRetryPolicy(self),
                 compression=None,
                 control_connection_timeout=self.connect_timeout,
-                connect_timeout=self.connect_timeout)
+                connect_timeout=self.connect_timeout,
+                idle_heartbeat_interval=0)
 
             self._session = cluster.connect(self.ks)
             self._session.default_timeout = None
@@ -1674,13 +1988,12 @@ class ImportProcess(ChildProcess):
 
     def run(self):
         try:
-            table_meta = self.session.cluster.metadata.keyspaces[self.ks].tables[self.table]
-            is_counter = ("counter" in [table_meta.columns[name].typestring for name in self.valid_columns])
+            pr = profile_on() if PROFILE_ON else None
 
-            if is_counter:
-                self.run_counter(table_meta)
-            else:
-                self.run_normal(table_meta)
+            self.inner_run(*self.make_params())
+
+            if pr:
+                profile_off(pr, file_name='worker_profile_%d.txt' % (os.getpid(),))
 
         except Exception, exc:
             if self.debug:
@@ -1694,67 +2007,88 @@ class ImportProcess(ChildProcess):
             self._session.cluster.shutdown()
         ChildProcess.close(self)
 
-    def run_counter(self, table_meta):
-        """
-        Main run method for tables that contain counter columns.
-        """
-        query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
-
-        # We prepare a query statement to find out the types of the partition key columns so we can
-        # route the update query to the correct replicas. As far as I understood this is the easiest
-        # way to find out the types of the partition columns, we will never use this prepared statement
-        where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
-        select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(self.ks), protect_name(self.table), where_clause)
-        conv = ImportConversion(self, table_meta, self.session.prepare(select_query))
-
-        while True:
-            batch = self.inmsg.get()
-            try:
-                for b in self.split_batches(batch, conv):
-                    self.send_counter_batch(query, conv, b)
+    def make_params(self):
+        metadata = self.session.cluster.metadata
+        table_meta = metadata.keyspaces[self.ks].tables[self.table]
+
+        prepared_statement = None
+        is_counter = ("counter" in [table_meta.columns[name].typestring for name in self.valid_columns])
+        if is_counter:
+            query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
+            make_statement = self.wrap_make_statement(self.make_counter_batch_statement)
+        elif self.use_prepared_statements:
+            query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
+                                                            protect_name(self.table),
+                                                            ', '.join(protect_names(self.valid_columns),),
+                                                            ', '.join(['?' for _ in self.valid_columns]))
+
+            query = self.session.prepare(query)
+            query.consistency_level = self.consistency_level
+            prepared_statement = query
+            make_statement = self.wrap_make_statement(self.make_prepared_batch_statement)
+        else:
+            query = 'INSERT INTO %s.%s (%s) VALUES (%%s)' % (protect_name(self.ks),
+                                                             protect_name(self.table),
+                                                             ', '.join(protect_names(self.valid_columns),))
+            make_statement = self.wrap_make_statement(self.make_non_prepared_batch_statement)
 
-            except Exception, exc:
-                self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                if self.debug:
-                    traceback.print_exc(exc)
+        conv = ImportConversion(self, table_meta, prepared_statement)
+        tm = TokenMap(self.ks, self.hostname, self.local_dc, self.session)
+        return query, conv, tm, make_statement
 
-    def run_normal(self, table_meta):
+    def inner_run(self, query, conv, tm, make_statement):
         """
-        Main run method for normal tables, i.e. tables that do not contain counter columns.
+        Main run method. Note that we bind self methods that are called inside loops
+        for performance reasons.
         """
-        query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
-                                                        protect_name(self.table),
-                                                        ', '.join(protect_names(self.valid_columns),),
-                                                        ', '.join(['?' for _ in self.valid_columns]))
+        self.query = query
+        self.conv = conv
+        self.make_statement = make_statement
 
-        query_statement = self.session.prepare(query)
-        query_statement.consistency_level = self.consistency_level
-        conv = ImportConversion(self, table_meta, query_statement)
+        convert_rows = self.convert_rows
+        split_into_batches = self.split_into_batches
+        result_callback = self.result_callback
+        err_callback = self.err_callback
+        session = self.session
 
         while True:
-            batch = self.inmsg.get()
+            chunk = self.inmsg.recv()
+            if chunk is None:
+                break
+
             try:
-                for b in self.split_batches(batch, conv):
-                    self.send_normal_batch(conv, query_statement, b)
+                chunk['rows'] = convert_rows(conv, chunk)
+                for replicas, batch in split_into_batches(chunk, conv, tm):
+                    statement = make_statement(query, conv, chunk, batch, replicas)
+                    future = session.execute_async(statement)
+                    future.add_callbacks(callback=result_callback, callback_args=(batch, chunk),
+                                         errback=err_callback, errback_args=(batch, chunk, replicas))
 
             except Exception, exc:
-                self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                if self.debug:
-                    traceback.print_exc(exc)
+                self.report_error(exc, chunk, chunk['rows'])
 
-    def send_counter_batch(self, query_text, conv, batch):
-        if self.test_failures and self.maybe_inject_failures(batch):
-            return
+    def wrap_make_statement(self, inner_make_statement):
+        def make_statement(query, conv, chunk, batch, replicas):
+            try:
+                return inner_make_statement(query, conv, batch, replicas)
+            except Exception, exc:
+                print "Failed to make batch statement: {}".format(exc)
+                self.report_error(exc, chunk, batch['rows'])
+                return None
 
-        error_rows = []
-        batch_statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+        def make_statement_with_failures(query, conv, chunk, batch, replicas):
+            failed_batch = self.maybe_inject_failures(batch)
+            if failed_batch:
+                return failed_batch
+            return make_statement(query, conv, chunk, batch, replicas)
 
-        for r in batch['rows']:
-            row = self.filter_row_values(r)
-            if len(row) != len(self.valid_columns):
-                error_rows.append(row)
-                continue
+        return make_statement_with_failures if self.test_failures else make_statement
 
+    def make_counter_batch_statement(self, query, conv, batch, replicas):
+        statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+        statement.replicas = replicas
+        statement.keyspace = self.ks
+        for row in batch['rows']:
             where_clause = []
             set_clause = []
             for i, value in enumerate(row):
@@ -1763,65 +2097,61 @@ class ImportProcess(ChildProcess):
                 else:
                     set_clause.append("%s=%s+%s" % (self.valid_columns[i], self.valid_columns[i], value))
 
-            full_query_text = query_text % (','.join(set_clause), ' AND '.join(where_clause))
-            batch_statement.add(full_query_text)
-
-        self.execute_statement(batch_statement, batch)
+            full_query_text = query % (','.join(set_clause), ' AND '.join(where_clause))
+            statement.add(full_query_text)
+        return statement
 
-        if error_rows:
-            self.outmsg.put((ImportTask.split_batch(batch, error_rows),
-                            '%s - %s' % (ParseError.__name__, "Failed to parse one or more rows")))
+    def make_prepared_batch_statement(self, query, _, batch, replicas):
+        """
+        Return a batch statement. This is an optimized version of:
 
-    def send_normal_batch(self, conv, query_statement, batch):
-        if self.test_failures and self.maybe_inject_failures(batch):
-            return
+            statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+            for row in batch['rows']:
+                statement.add(query, row)
 
-        good_rows, converted_rows, errors = self.convert_rows(conv, batch['rows'])
+        We could optimize further by removing bound_statements altogether but we'd have to duplicate much
+        more driver's code (BoundStatement.bind()).
+        """
+        statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+        statement.replicas = replicas
+        statement.keyspace = self.ks
+        statement._statements_and_parameters = [(True, query.query_id, query.bind(r).values) for r in batch['rows']]
+        return statement
 
-        if converted_rows:
-            try:
-                statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
-                for row in converted_rows:
-                    statement.add(query_statement, row)
-                self.execute_statement(statement, ImportTask.split_batch(batch, good_rows))
-            except Exception, exc:
-                self.err_callback(exc, ImportTask.split_batch(batch, good_rows))
+    def make_non_prepared_batch_statement(self, query, _, batch, replicas):
+        statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+        statement.replicas = replicas
+        statement.keyspace = self.ks
+        statement._statements_and_parameters = [(False, query % (','.join(r),), ()) for r in batch['rows']]
+        return statement
 
-        if errors:
-            for msg, rows in errors.iteritems():
-                self.outmsg.put((ImportTask.split_batch(batch, rows),
-                                '%s - %s' % (ParseError.__name__, msg)))
-
-    def convert_rows(self, conv, rows):
+    def convert_rows(self, conv, chunk):
         """
-        Try to convert each row. If conversion is OK then add the converted result to converted_rows
-        and the original string to good_rows. Else add the original string to error_rows. Return the three
-        arrays.
+        Return converted rows and report any errors during conversion.
         """
-        good_rows = []
-        errors = defaultdict(list)
-        converted_rows = []
+        def filter_row_values(row):
+            return [v for i, v in enumerate(row) if i not in self.skip_column_indexes]
 
-        for r in rows:
-            row = self.filter_row_values(r)
-            if len(row) != len(self.valid_columns):
-                msg = 'Invalid row length %d should be %d' % (len(row), len(self.valid_columns))
-                errors[msg].append(row)
-                continue
+        if self.skip_column_indexes:
+            rows = [filter_row_values(r) for r in list(csv.reader(chunk['rows'], **self.dialect_options))]
+        else:
+            rows = list(csv.reader(chunk['rows'], **self.dialect_options))
 
-            try:
-                converted_rows.append(conv.get_row_values(row))
-                good_rows.append(row)
-            except ParseError, err:
-                errors[err.message].append(row)
+        errors = defaultdict(list)
 
-        return good_rows, converted_rows, errors
+        def convert_row(r):
+            try:
+                return conv.convert_row(r)
+            except Exception, err:
+                errors[err.message].append(r)
+                return None
 
-    def filter_row_values(self, row):
-        if not self.skip_column_indexes:
-            return row
+        converted_rows = filter(None, [convert_row(r) for r in rows])
 
-        return [v for i, v in enumerate(row) if i not in self.skip_column_indexes]
+        if errors:
+            for msg, rows in errors.iteritems():
+                self.report_error(ParseError(msg), chunk, rows)
+        return converted_rows
 
     def maybe_inject_failures(self, batch):
         """
@@ -1836,86 +2166,94 @@ class ImportProcess(ChildProcess):
                 if batch['attempts'] < failing_batch['failures']:
                     statement = SimpleStatement("INSERT INTO badtable (a, b) VALUES (1, 2)",
                                                 consistency_level=self.consistency_level)
-                    self.execute_statement(statement, batch)
-                    return True
+                    return statement
 
         if 'exit_batch' in self.test_failures:
             exit_batch = self.test_failures['exit_batch']
             if exit_batch['id'] == batch['id']:
                 sys.exit(1)
 
-        return False  # carry on as normal
+        return None  # carry on as normal
 
-    def execute_statement(self, statement, batch):
-        future = self.session.execute_async(statement)
-        future.add_callbacks(callback=self.result_callback, callback_args=(batch, ),
-                             errback=self.err_callback, errback_args=(batch, ))
+    @staticmethod
+    def make_batch(batch_id, rows, attempts=1):
+        return {'id': batch_id, 'rows': rows, 'attempts': attempts}
 
-    def split_batches(self, batch, conv):
+    def split_into_batches(self, chunk, conv, tm):
         """
-        Batch rows by partition key, if there are at least min_batch_size (2)
-        rows with the same partition key. These batches can be as big as they want
-        since this translates to a single insert operation server side.
-
-        If there are less than min_batch_size rows for a partition, work out the
-        first replica for this partition and add the rows to replica left-over rows.
-
-        Then batch the left-overs of each replica up to max_batch_size.
+        Batch rows by ring position or replica.
+        If there are at least min_batch_size rows for a ring position then split these rows into
+        groups of max_batch_size and send a batch for each group, using all replicas for this ring position.
+        Otherwise, we are forced to batch by replica, and here unfortunately we can only choose one replica to
+        guarantee common replicas across partition keys. We are typically able
+        to batch by ring position for small clusters or when VNODES are not used. For large clusters with VNODES
+        it may not be possible, in this case it helps to increase the CHUNK SIZE but up to a limit, otherwise
+        we may choke the cluster.
         """
-        rows_by_pk = defaultdict(list)
+
+        rows_by_ring_pos = defaultdict(list)
         errors = defaultdict(list)
 
-        for row in batch['rows']:
+        min_batch_size = self.min_batch_size
+        max_batch_size = self.max_batch_size
+        ring = tm.ring
+
+        get_row_partition_key_values = conv.get_row_partition_key_values_fcn()
+        pk_to_token_value = tm.pk_to_token_value
+        get_ring_pos = tm.get_ring_pos
+        make_batch = self.make_batch
+
+        for row in chunk['rows']:
             try:
-                pk = conv.get_row_partition_key_values(row)
-                rows_by_pk[pk].append(row)
-            except ParseError, e:
+                pk = get_row_partition_key_values(row)
+                rows_by_ring_pos[get_ring_pos(ring, pk_to_token_value(pk))].append(row)
+            except Exception, e:
                 errors[e.message].append(row)
 
         if errors:
             for msg, rows in errors.iteritems():
-                self.outmsg.put((ImportTask.split_batch(batch, rows),
-                                 '%s - %s' % (ParseError.__name__, msg)))
+                self.report_error(ParseError(msg), chunk, rows)
 
+        replicas = tm.replicas
+        filter_replicas = tm.filter_replicas
         rows_by_replica = defaultdict(list)
-        for pk, rows in rows_by_pk.iteritems():
-            if len(rows) >= self.min_batch_size:
-                yield ImportTask.make_batch(batch['id'], rows, batch['attempts'])
+        for ring_pos, rows in rows_by_ring_pos.iteritems():
+            if len(rows) > min_batch_size:
+                for i in xrange(0, len(rows), max_batch_size):
+                    yield filter_replicas(replicas[ring_pos]), make_batch(chunk['id'], rows[i:i + max_batch_size])
             else:
-                replica = self.get_replica(pk)
-                rows_by_replica[replica].extend(rows)
-
-        for replica, rows in rows_by_replica.iteritems():
-            for b in self.batches(rows, batch):
-                yield b
-
-    def get_replica(self, pk):
-        """
-        Return the first replica or the host we are already connected to if there are no local
-        replicas that are up. We always use the first replica to match the replica chosen by the driver
-        TAR, see TokenAwarePolicy.make_query_plan().
-        """
-        metadata = self.session.cluster.metadata
-        replicas = filter(lambda r: r.is_up and r.datacenter == self.local_dc, metadata.get_replicas(self.ks, pk))
-        ret = replicas[0].address if len(replicas) > 0 else self.hostname
-        return ret
-
-    def batches(self, rows, batch):
-        """
-        Split rows into batches of max_batch_size
-        """
-        for i in xrange(0, len(rows), self.max_batch_size):
-            yield ImportTask.make_batch(batch['id'], rows[i:i + self.max_batch_size], batch['attempts'])
-
-    def result_callback(self, _, batch):
-        batch['imported'] = len(batch['rows'])
-        batch['rows'] = []  # no need to resend these, just send the count in 'imported'
-        self.outmsg.put((batch, None))
-
-    def err_callback(self, response, batch):
-        self.outmsg.put((batch, '%s - %s' % (response.__class__.__name__, response.message)))
+                # select only the first valid replica to guarantee more overlap or none at all
+                rows_by_replica[filter_replicas(replicas[ring_pos])[:1]].extend(rows)
+
+        # Now send the batches by replica
+        for replicas, rows in rows_by_replica.iteritems():
+            for i in xrange(0, len(rows), max_batch_size):
+                yield replicas, make_batch(chunk['id'], rows[i:i + max_batch_size])
+
+    def result_callback(self, _, batch, chunk):
+        self.update_chunk(batch['rows'], chunk)
+
+    def err_callback(self, response, batch, chunk, replicas):
+        err_is_final = batch['attempts'] >= self.max_attempts
+        self.report_error(response, chunk, batch['rows'], batch['attempts'], err_is_final)
+        if not err_is_final:
+            batch['attempts'] += 1
+            statement = self.make_statement(self.query, self.conv, chunk, batch, replicas)
+            future = self.session.execute_async(statement)
+            future.add_callbacks(callback=self.result_callback, callback_args=(batch, chunk),
+                                 errback=self.err_callback, errback_args=(batch, chunk, replicas))
+
+    def report_error(self, err, chunk, rows=None, attempts=1, final=True):
         if self.debug:
-            traceback.print_exc(response)
+            traceback.print_exc(err)
+        self.outmsg.send(ImportTaskError(err.__class__.__name__, err.message, rows, attempts, final))
+        if final:
+            self.update_chunk(rows, chunk)
+
+    def update_chunk(self, rows, chunk):
+        chunk['imported'] += len(rows)
+        if chunk['imported'] == chunk['num_rows_sent']:
+            self.outmsg.send(ImportProcessResult(chunk['num_rows_sent']))
 
 
 class RateMeter(object):
@@ -1937,11 +2275,19 @@ class RateMeter(object):
         self.current_record += n
         self.maybe_update()
 
-    def maybe_update(self):
+    def maybe_update(self, sleep=False):
+        if self.current_record == 0:
+            return
+
         new_checkpoint_time = time.time()
-        if new_checkpoint_time - self.last_checkpoint_time >= self.update_interval:
+        time_difference = new_checkpoint_time - self.last_checkpoint_time
+        if time_difference >= self.update_interval:
             self.update(new_checkpoint_time)
             self.log_message()
+        elif sleep:
+            remaining_time = time_difference - self.update_interval
+            if remaining_time > 0.000001:
+                time.sleep(remaining_time)
 
     def update(self, new_checkpoint_time):
         time_difference = new_checkpoint_time - self.last_checkpoint_time

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/pylib/cqlshlib/util.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/util.py b/pylib/cqlshlib/util.py
index 281aad6..3ee128d 100644
--- a/pylib/cqlshlib/util.py
+++ b/pylib/cqlshlib/util.py
@@ -23,6 +23,12 @@ from itertools import izip
 from datetime import timedelta, tzinfo
 from StringIO import StringIO
 
+try:
+    from line_profiler import LineProfiler
+    HAS_LINE_PROFILER = True
+except ImportError:
+    HAS_LINE_PROFILER = False
+
 ZERO = timedelta(0)
 
 
@@ -126,18 +132,35 @@ def get_file_encoding_bomsize(filename):
     else:
         file_encoding, size = "utf-8", 0
 
-    return (file_encoding, size)
+    return file_encoding, size
+
 
+def profile_on(fcn_names=None):
+    if fcn_names and HAS_LINE_PROFILER:
+        pr = LineProfiler()
+        for fcn_name in fcn_names:
+            pr.add_function(fcn_name)
+        pr.enable()
+        return pr
 
-def profile_on():
     pr = cProfile.Profile()
     pr.enable()
     return pr
 
 
-def profile_off(pr):
+def profile_off(pr, file_name):
     pr.disable()
     s = StringIO()
-    ps = pstats.Stats(pr, stream=s).sort_stats('cumulative')
-    ps.print_stats()
-    print s.getvalue()
+
+    if HAS_LINE_PROFILER and isinstance(pr, LineProfiler):
+        pr.print_stats(s)
+    else:
+        ps = pstats.Stats(pr, stream=s).sort_stats('cumulative')
+        ps.print_stats()
+
+    ret = s.getvalue()
+    if file_name:
+        with open(file_name, 'w') as f:
+            print "Writing to %s\n" % (f.name, )
+            f.write(ret)
+    return ret

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3d2f26f/pylib/setup.py
----------------------------------------------------------------------
diff --git a/pylib/setup.py b/pylib/setup.py
index 704d077..3654502 100755
--- a/pylib/setup.py
+++ b/pylib/setup.py
@@ -16,9 +16,11 @@
 # limitations under the License.
 
 from distutils.core import setup
+from Cython.Build import cythonize
 
 setup(
     name="cassandra-pylib",
     description="Cassandra Python Libraries",
     packages=["cqlshlib"],
+    ext_modules=cythonize("cqlshlib/copyutil.py"),
 )


[20/23] cassandra git commit: Merge commit 'b74ffeafd2e08a7669013d6b4ba87980e1ad5379' into cassandra-3.0

Posted by sl...@apache.org.
Merge commit 'b74ffeafd2e08a7669013d6b4ba87980e1ad5379' into cassandra-3.0


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/49c616cf
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/49c616cf
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/49c616cf

Branch: refs/heads/cassandra-3.0
Commit: 49c616cf0fc9c8d4649e2ec71a07fb9fd7831318
Parents: 34b07a7 b74ffea
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue Mar 8 10:23:48 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Mar 8 10:24:26 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                |    2 +-
 bin/cqlsh.py               |   33 +-
 pylib/cqlshlib/copyutil.py | 1168 +++++++++++++++++++++++++--------------
 pylib/cqlshlib/util.py     |   35 +-
 pylib/setup.py             |    2 +
 5 files changed, 821 insertions(+), 419 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/49c616cf/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 38bf2ee,7b67cdc..5b92143
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,39 -1,11 +1,39 @@@
 -2.2.6
 +3.0.5
 + * Fix filtering on non-primary key columns for queries without index (CASSANDRA-6377)
 + * Fix sstableloader fail when using materialized view (CASSANDRA-11275)
 +Merged from 2.2:
   * Fix AE in nodetool cfstats (backport CASSANDRA-10859) (CASSANDRA-11297)
   * Unresolved hostname leads to replace being ignored (CASSANDRA-11210)
 - * Fix filtering on non-primary key columns for thrift static column families
 -   (CASSANDRA-6377)
   * Only log yaml config once, at startup (CASSANDRA-11217)
 - * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
   * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
 +Merged from 2.1:
 + * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
- 
++ * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
 +
 +3.0.4
 + * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
 + * MV should only query complex columns included in the view (CASSANDRA-11069)
 + * Failed aggregate creation breaks server permanently (CASSANDRA-11064)
 + * Add sstabledump tool (CASSANDRA-7464)
 + * Introduce backpressure for hints (CASSANDRA-10972)
 + * Fix ClusteringPrefix not being able to read tombstone range boundaries (CASSANDRA-11158)
 + * Prevent logging in sandboxed state (CASSANDRA-11033)
 + * Disallow drop/alter operations of UDTs used by UDAs (CASSANDRA-10721)
 + * Add query time validation method on Index (CASSANDRA-11043)
 + * Avoid potential AssertionError in mixed version cluster (CASSANDRA-11128)
 + * Properly handle hinted handoff after topology changes (CASSANDRA-5902)
 + * AssertionError when listing sstable files on inconsistent disk state (CASSANDRA-11156)
 + * Fix wrong rack counting and invalid conditions check for TokenAllocation
 +   (CASSANDRA-11139)
 + * Avoid creating empty hint files (CASSANDRA-11090)
 + * Fix leak detection strong reference loop using weak reference (CASSANDRA-11120)
 + * Configurie BatchlogManager to stop delayed tasks on shutdown (CASSANDRA-11062)
 + * Hadoop integration is incompatible with Cassandra Driver 3.0.0 (CASSANDRA-11001)
 + * Add dropped_columns to the list of schema table so it gets handled
 +   properly (CASSANDRA-11050)
 + * Fix NPE when using forceRepairRangeAsync without DC (CASSANDRA-11239)
 +Merged from 2.2:
 + * Unresolved hostname leads to replace being ignored (CASSANDRA-11210)
   * Range.compareTo() violates the contract of Comparable (CASSANDRA-11216)
   * Avoid NPE when serializing ErrorMessage with null message (CASSANDRA-11167)
   * Replacing an aggregate with a new version doesn't reset INITCOND (CASSANDRA-10840)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/49c616cf/bin/cqlsh.py
----------------------------------------------------------------------
diff --cc bin/cqlsh.py
index 9dd9634,9082d72..83dbeed
--- a/bin/cqlsh.py
+++ b/bin/cqlsh.py
@@@ -602,24 -593,25 +602,44 @@@ def insert_driver_hooks()
  
  
  def extend_cql_deserialization():
-     # The python driver returns BLOBs as string, but we expect them as bytearrays
+     """
 -    The python driver returns BLOBs as string, but we expect them as bytearrays; therefore we change
++    The python driver returns BLOBs as string, but we expect them as bytearrays
+     the implementation of cassandra.cqltypes.BytesType.deserialize.
+ 
+     The deserializers package exists only when the driver has been compiled with cython extensions and
+     cassandra.deserializers.DesBytesType replaces cassandra.cqltypes.BytesType.deserialize.
+ 
+     DesBytesTypeByteArray is a fast deserializer that converts blobs into bytearrays but it was
+     only introduced recently (3.1.0). If it is available we use it, otherwise we remove
+     cassandra.deserializers.DesBytesType so that we fall back onto cassandra.cqltypes.BytesType.deserialize
+     just like in the case where no cython extensions are present.
+     """
+     if hasattr(cassandra, 'deserializers'):
+         if hasattr(cassandra.deserializers, 'DesBytesTypeByteArray'):
+             cassandra.deserializers.DesBytesType = cassandra.deserializers.DesBytesTypeByteArray
+         else:
+             del cassandra.deserializers.DesBytesType
+ 
      cassandra.cqltypes.BytesType.deserialize = staticmethod(lambda byts, protocol_version: bytearray(byts))
 +
 +    class DateOverFlowWarning(RuntimeWarning):
 +        pass
 +
 +    # Native datetime types blow up outside of datetime.[MIN|MAX]_YEAR. We will fall back to an int timestamp
 +    def deserialize_date_fallback_int(byts, protocol_version):
 +        timestamp_ms = int64_unpack(byts)
 +        try:
 +            return datetime_from_timestamp(timestamp_ms / 1000.0)
 +        except OverflowError:
 +            warnings.warn(DateOverFlowWarning("Some timestamps are larger than Python datetime can represent. Timestamps are displayed in milliseconds from epoch."))
 +            return timestamp_ms
 +
 +    cassandra.cqltypes.DateType.deserialize = staticmethod(deserialize_date_fallback_int)
 +
++    if hasattr(cassandra, 'deserializers'):
++        del cassandra.deserializers.DesDateType
++
 +    # Return cassandra.cqltypes.EMPTY instead of None for empty values
      cassandra.cqltypes.CassandraType.support_empty_values = True
  
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/49c616cf/pylib/cqlshlib/copyutil.py
----------------------------------------------------------------------
diff --cc pylib/cqlshlib/copyutil.py
index aeb2d0b,2755dd5..6be990d
--- a/pylib/cqlshlib/copyutil.py
+++ b/pylib/cqlshlib/copyutil.py
@@@ -1484,7 -1714,7 +1714,13 @@@ class ImportConversion(object)
  
              m = p.match(val)
              if not m:
--                raise ValueError("can't interpret %r as a date with this format: %s" % (val, self.date_time_format))
++                try:
++                    # in case of overflow COPY TO prints dates as milliseconds from the epoch, see
++                    # deserialize_date_fallback_int in cqlsh.py
++                    return int(val)
++                except ValueError:
++                    raise ValueError("can't interpret %r as a date with format %s or as int" % (val,
++                                                                                                self.date_time_format))
  
              # https://docs.python.org/2/library/time.html#time.struct_time
              tval = time.struct_time((int(m.group(1)), int(m.group(2)), int(m.group(3)),  # year, month, day


[21/23] cassandra git commit: Merge commit '49c616cf0fc9c8d4649e2ec71a07fb9fd7831318' into cassandra-3.5

Posted by sl...@apache.org.
Merge commit '49c616cf0fc9c8d4649e2ec71a07fb9fd7831318' into cassandra-3.5


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6329d54a
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6329d54a
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6329d54a

Branch: refs/heads/cassandra-3.5
Commit: 6329d54a6c802e703902ee6cf842e983820d144b
Parents: c9e9b62 49c616c
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue Mar 8 10:25:32 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Mar 8 10:25:54 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                |    1 +
 bin/cqlsh.py               |   33 +-
 pylib/cqlshlib/copyutil.py | 1173 +++++++++++++++++++++++++--------------
 pylib/cqlshlib/util.py     |   35 +-
 pylib/setup.py             |    2 +
 5 files changed, 824 insertions(+), 420 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6329d54a/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index c791139,5b92143..c88ae54
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -7,42 -7,11 +7,43 @@@ Merged from 2.2
   * Only log yaml config once, at startup (CASSANDRA-11217)
   * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
  Merged from 2.1:
 - * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
+  * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
 + * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
  
 -3.0.4
 - * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
 +
 +3.4
 + * (cqlsh) add cqlshrc option to always connect using ssl (CASSANDRA-10458)
 + * Cleanup a few resource warnings (CASSANDRA-11085)
 + * Allow custom tracing implementations (CASSANDRA-10392)
 + * Extract LoaderOptions to be able to be used from outside (CASSANDRA-10637)
 + * fix OnDiskIndexTest to properly treat empty ranges (CASSANDRA-11205)
 + * fix TrackerTest to handle new notifications (CASSANDRA-11178)
 + * add SASI validation for partitioner and complex columns (CASSANDRA-11169)
 + * Add caching of encrypted credentials in PasswordAuthenticator (CASSANDRA-7715)
 + * fix SASI memtable switching on flush (CASSANDRA-11159)
 + * Remove duplicate offline compaction tracking (CASSANDRA-11148)
 + * fix EQ semantics of analyzed SASI indexes (CASSANDRA-11130)
 + * Support long name output for nodetool commands (CASSANDRA-7950)
 + * Encrypted hints (CASSANDRA-11040)
 + * SASI index options validation (CASSANDRA-11136)
 + * Optimize disk seek using min/max column name meta data when the LIMIT clause is used
 +   (CASSANDRA-8180)
 + * Add LIKE support to CQL3 (CASSANDRA-11067)
 + * Generic Java UDF types (CASSANDRA-10819)
 + * cqlsh: Include sub-second precision in timestamps by default (CASSANDRA-10428)
 + * Set javac encoding to utf-8 (CASSANDRA-11077)
 + * Integrate SASI index into Cassandra (CASSANDRA-10661)
 + * Add --skip-flush option to nodetool snapshot
 + * Skip values for non-queried columns (CASSANDRA-10657)
 + * Add support for secondary indexes on static columns (CASSANDRA-8103)
 + * CommitLogUpgradeTestMaker creates broken commit logs (CASSANDRA-11051)
 + * Add metric for number of dropped mutations (CASSANDRA-10866)
 + * Simplify row cache invalidation code (CASSANDRA-10396)
 + * Support user-defined compaction through nodetool (CASSANDRA-10660)
 + * Stripe view locks by key and table ID to reduce contention (CASSANDRA-10981)
 + * Add nodetool gettimeout and settimeout commands (CASSANDRA-10953)
 + * Add 3.0 metadata to sstablemetadata output (CASSANDRA-10838)
 +Merged from 3.0:
   * MV should only query complex columns included in the view (CASSANDRA-11069)
   * Failed aggregate creation breaks server permanently (CASSANDRA-11064)
   * Add sstabledump tool (CASSANDRA-7464)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6329d54a/bin/cqlsh.py
----------------------------------------------------------------------
diff --cc bin/cqlsh.py
index 78fedeb,83dbeed..d007d75
--- a/bin/cqlsh.py
+++ b/bin/cqlsh.py
@@@ -480,7 -475,7 +480,7 @@@ COPY_COMMON_OPTIONS = ['DELIMITER', 'QU
                         'MAXATTEMPTS', 'REPORTFREQUENCY', 'DECIMALSEP', 'THOUSANDSSEP', 'BOOLSTYLE',
                         'NUMPROCESSES', 'CONFIGFILE', 'RATEFILE']
  COPY_FROM_OPTIONS = ['CHUNKSIZE', 'INGESTRATE', 'MAXBATCHSIZE', 'MINBATCHSIZE', 'MAXROWS',
-                      'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE', 'TTL']
 -                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE', 'PREPAREDSTATEMENTS']
++                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE', 'PREPAREDSTATEMENTS', 'TTL']
  COPY_TO_OPTIONS = ['ENCODING', 'PAGESIZE', 'PAGETIMEOUT', 'BEGINTOKEN', 'ENDTOKEN', 'MAXOUTPUTSIZE', 'MAXREQUESTS']
  
  
@@@ -1865,7 -1884,11 +1885,12 @@@ class Shell(cmd.Cmd)
            MAXINSERTERRORS=-1      - the maximum global number of insert errors, -1 means no maximum
            ERRFILE=''              - a file where to store all rows that could not be imported, by default this is
                                      import_ks_table.err where <ks> is your keyspace and <table> is your table name.
+           PREPAREDSTATEMENTS=True - whether to use prepared statements when importing, by default True. Set this to
+                                     False if you don't mind shifting data parsing to the cluster. The cluster will also
+                                     have to compile every batch statement. For large and oversized clusters
+                                     this will result in a faster import but for smaller clusters it may generate
+                                     timeouts.
 +          TTL=3600                - the time to live in seconds, by default data will not expire
  
          Available COPY TO options and defaults:
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6329d54a/pylib/cqlshlib/copyutil.py
----------------------------------------------------------------------
diff --cc pylib/cqlshlib/copyutil.py
index 95da679,6be990d..e690e82
--- a/pylib/cqlshlib/copyutil.py
+++ b/pylib/cqlshlib/copyutil.py
@@@ -187,7 -275,7 +275,8 @@@ class CopyTask(object)
          copy_options['errfile'] = safe_normpath(opts.pop('errfile', 'import_%s_%s.err' % (self.ks, self.table,)))
          copy_options['ratefile'] = safe_normpath(opts.pop('ratefile', ''))
          copy_options['maxoutputsize'] = int(opts.pop('maxoutputsize', '-1'))
+         copy_options['preparedstatements'] = bool(opts.pop('preparedstatements', 'true').lower() == 'true')
 +        copy_options['ttl'] = int(opts.pop('ttl', -1))
  
          self.check_options(copy_options)
          return CopyOptions(copy=copy_options, dialect=dialect_options, unrecognized=opts)
@@@ -1655,8 -1965,12 +1969,13 @@@ class ImportProcess(ChildProcess)
          self.max_attempts = options.copy['maxattempts']
          self.min_batch_size = options.copy['minbatchsize']
          self.max_batch_size = options.copy['maxbatchsize']
+         self.use_prepared_statements = options.copy['preparedstatements']
 +        self.ttl = options.copy['ttl']
+         self.dialect_options = options.dialect
          self._session = None
+         self.query = None
+         self.conv = None
+         self.make_statement = None
  
      @property
      def session(self):
@@@ -1700,69 -2014,88 +2019,91 @@@
              self._session.cluster.shutdown()
          ChildProcess.close(self)
  
-     def run_counter(self, table_meta):
-         """
-         Main run method for tables that contain counter columns.
-         """
-         query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
- 
-         # We prepare a query statement to find out the types of the partition key columns so we can
-         # route the update query to the correct replicas. As far as I understood this is the easiest
-         # way to find out the types of the partition columns, we will never use this prepared statement
-         where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
-         select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(self.ks), protect_name(self.table), where_clause)
-         conv = ImportConversion(self, table_meta, self.session.prepare(select_query))
- 
-         while True:
-             batch = self.inmsg.get()
-             try:
-                 for b in self.split_batches(batch, conv):
-                     self.send_counter_batch(query, conv, b)
+     def make_params(self):
+         metadata = self.session.cluster.metadata
+         table_meta = metadata.keyspaces[self.ks].tables[self.table]
+ 
+         prepared_statement = None
+         is_counter = ("counter" in [table_meta.columns[name].cql_type for name in self.valid_columns])
+         if is_counter:
+             query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
+             make_statement = self.wrap_make_statement(self.make_counter_batch_statement)
+         elif self.use_prepared_statements:
+             query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
+                                                             protect_name(self.table),
+                                                             ', '.join(protect_names(self.valid_columns),),
+                                                             ', '.join(['?' for _ in self.valid_columns]))
 -
++            if self.ttl >= 0:
++                query += 'USING TTL %s' % (self.ttl,)
+             query = self.session.prepare(query)
+             query.consistency_level = self.consistency_level
+             prepared_statement = query
+             make_statement = self.wrap_make_statement(self.make_prepared_batch_statement)
+         else:
+             query = 'INSERT INTO %s.%s (%s) VALUES (%%s)' % (protect_name(self.ks),
+                                                              protect_name(self.table),
+                                                              ', '.join(protect_names(self.valid_columns),))
++            if self.ttl >= 0:
++                query += 'USING TTL %s' % (self.ttl,)
+             make_statement = self.wrap_make_statement(self.make_non_prepared_batch_statement)
  
-             except Exception, exc:
-                 self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                 if self.debug:
-                     traceback.print_exc(exc)
+         conv = ImportConversion(self, table_meta, prepared_statement)
+         tm = TokenMap(self.ks, self.hostname, self.local_dc, self.session)
+         return query, conv, tm, make_statement
  
-     def run_normal(self, table_meta):
+     def inner_run(self, query, conv, tm, make_statement):
          """
-         Main run method for normal tables, i.e. tables that do not contain counter columns.
+         Main run method. Note that we bind self methods that are called inside loops
+         for performance reasons.
          """
-         query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
-                                                         protect_name(self.table),
-                                                         ', '.join(protect_names(self.valid_columns),),
-                                                         ', '.join(['?' for _ in self.valid_columns]))
-         if self.ttl >= 0:
-             query += 'USING TTL %s' % (self.ttl,)
+         self.query = query
+         self.conv = conv
+         self.make_statement = make_statement
  
-         query_statement = self.session.prepare(query)
-         query_statement.consistency_level = self.consistency_level
-         conv = ImportConversion(self, table_meta, query_statement)
+         convert_rows = self.convert_rows
+         split_into_batches = self.split_into_batches
+         result_callback = self.result_callback
+         err_callback = self.err_callback
+         session = self.session
  
          while True:
-             batch = self.inmsg.get()
+             chunk = self.inmsg.recv()
+             if chunk is None:
+                 break
+ 
              try:
-                 for b in self.split_batches(batch, conv):
-                     self.send_normal_batch(conv, query_statement, b)
+                 chunk['rows'] = convert_rows(conv, chunk)
+                 for replicas, batch in split_into_batches(chunk, conv, tm):
+                     statement = make_statement(query, conv, chunk, batch, replicas)
+                     future = session.execute_async(statement)
+                     future.add_callbacks(callback=result_callback, callback_args=(batch, chunk),
+                                          errback=err_callback, errback_args=(batch, chunk, replicas))
  
              except Exception, exc:
-                 self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                 if self.debug:
-                     traceback.print_exc(exc)
+                 self.report_error(exc, chunk, chunk['rows'])
  
-     def send_counter_batch(self, query_text, conv, batch):
-         if self.test_failures and self.maybe_inject_failures(batch):
-             return
+     def wrap_make_statement(self, inner_make_statement):
+         def make_statement(query, conv, chunk, batch, replicas):
+             try:
+                 return inner_make_statement(query, conv, batch, replicas)
+             except Exception, exc:
+                 print "Failed to make batch statement: {}".format(exc)
+                 self.report_error(exc, chunk, batch['rows'])
+                 return None
  
-         error_rows = []
-         batch_statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+         def make_statement_with_failures(query, conv, chunk, batch, replicas):
+             failed_batch = self.maybe_inject_failures(batch)
+             if failed_batch:
+                 return failed_batch
+             return make_statement(query, conv, chunk, batch, replicas)
  
-         for r in batch['rows']:
-             row = self.filter_row_values(r)
-             if len(row) != len(self.valid_columns):
-                 error_rows.append(row)
-                 continue
+         return make_statement_with_failures if self.test_failures else make_statement
  
+     def make_counter_batch_statement(self, query, conv, batch, replicas):
+         statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+         statement.replicas = replicas
+         statement.keyspace = self.ks
+         for row in batch['rows']:
              where_clause = []
              set_clause = []
              for i, value in enumerate(row):


[17/23] cassandra git commit: Merge commit 'c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc' into cassandra-2.2

Posted by sl...@apache.org.
Merge commit 'c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc' into cassandra-2.2


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b74ffeaf
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b74ffeaf
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b74ffeaf

Branch: refs/heads/cassandra-3.5
Commit: b74ffeafd2e08a7669013d6b4ba87980e1ad5379
Parents: 6e0395e c3d2f26
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue Mar 8 10:22:22 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Mar 8 10:23:08 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                |    1 +
 bin/cqlsh.py               |   28 +-
 pylib/cqlshlib/copyutil.py | 1160 +++++++++++++++++++++++++--------------
 pylib/cqlshlib/util.py     |   35 +-
 pylib/setup.py             |    2 +
 5 files changed, 809 insertions(+), 417 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b74ffeaf/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 89fa904,d6b085c..7b67cdc
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,30 -1,5 +1,31 @@@
 -2.1.14
 +2.2.6
 + * Fix AE in nodetool cfstats (backport CASSANDRA-10859) (CASSANDRA-11297)
 + * Unresolved hostname leads to replace being ignored (CASSANDRA-11210)
 + * Fix filtering on non-primary key columns for thrift static column families
 +   (CASSANDRA-6377)
 + * Only log yaml config once, at startup (CASSANDRA-11217)
 + * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
 + * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
 + * Range.compareTo() violates the contract of Comparable (CASSANDRA-11216)
 + * Avoid NPE when serializing ErrorMessage with null message (CASSANDRA-11167)
 + * Replacing an aggregate with a new version doesn't reset INITCOND (CASSANDRA-10840)
 + * (cqlsh) cqlsh cannot be called through symlink (CASSANDRA-11037)
 + * fix ohc and java-driver pom dependencies in build.xml (CASSANDRA-10793)
 + * Protect from keyspace dropped during repair (CASSANDRA-11065)
 + * Handle adding fields to a UDT in SELECT JSON and toJson() (CASSANDRA-11146)
 + * Better error message for cleanup (CASSANDRA-10991)
 + * cqlsh pg-style-strings broken if line ends with ';' (CASSANDRA-11123)
 + * Use cloned TokenMetadata in size estimates to avoid race against membership check
 +   (CASSANDRA-10736)
 + * Always persist upsampled index summaries (CASSANDRA-10512)
 + * (cqlsh) Fix inconsistent auto-complete (CASSANDRA-10733)
 + * Make SELECT JSON and toJson() threadsafe (CASSANDRA-11048)
 + * Fix SELECT on tuple relations for mixed ASC/DESC clustering order (CASSANDRA-7281)
 + * (cqlsh) Support utf-8/cp65001 encoding on Windows (CASSANDRA-11030)
 + * Fix paging on DISTINCT queries repeats result when first row in partition changes
 +   (CASSANDRA-10010)
 +Merged from 2.1:
+  * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
   * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
   * Don't remove FailureDetector history on removeEndpoint (CASSANDRA-10371)
   * Only notify if repair status changed (CASSANDRA-11172)


[06/23] cassandra git commit: Merge commit 'c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc' into cassandra-2.2

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b74ffeaf/pylib/cqlshlib/copyutil.py
----------------------------------------------------------------------
diff --cc pylib/cqlshlib/copyutil.py
index aeb2d0b,cd03765..2755dd5
--- a/pylib/cqlshlib/copyutil.py
+++ b/pylib/cqlshlib/copyutil.py
@@@ -45,9 -51,13 +51,13 @@@ from cassandra.util import Date, Tim
  
  from cql3handling import CqlRuleSet
  from displaying import NO_COLOR_MAP
 -from formatting import format_value_default, EMPTY, get_formatter
 +from formatting import format_value_default, DateTimeFormat, EMPTY, get_formatter
  from sslhandling import ssl_settings
  
+ PROFILE_ON = False
+ STRACE_ON = False
+ IS_LINUX = platform.system() == 'Linux'
+ 
  CopyOptions = namedtuple('CopyOptions', 'copy dialect unrecognized')
  
  
@@@ -164,13 -252,12 +252,13 @@@ class CopyTask(object)
          # in the page size or 10 seconds if pagesize is smaller
          copy_options['pagetimeout'] = int(opts.pop('pagetimeout', max(10, 10 * (copy_options['pagesize'] / 1000))))
          copy_options['maxattempts'] = int(opts.pop('maxattempts', 5))
 -        copy_options['dtformats'] = opts.pop('datetimeformat', shell.display_time_format)
 +        copy_options['dtformats'] = DateTimeFormat(opts.pop('datetimeformat', shell.display_timestamp_format),
 +                                                   shell.display_date_format, shell.display_nanotime_format)
          copy_options['float_precision'] = shell.display_float_precision
-         copy_options['chunksize'] = int(opts.pop('chunksize', 1000))
+         copy_options['chunksize'] = int(opts.pop('chunksize', 5000))
          copy_options['ingestrate'] = int(opts.pop('ingestrate', 100000))
          copy_options['maxbatchsize'] = int(opts.pop('maxbatchsize', 20))
-         copy_options['minbatchsize'] = int(opts.pop('minbatchsize', 2))
+         copy_options['minbatchsize'] = int(opts.pop('minbatchsize', 10))
          copy_options['reportfrequency'] = float(opts.pop('reportfrequency', 0.25))
          copy_options['consistencylevel'] = shell.consistency_level
          copy_options['decimalsep'] = opts.pop('decimalsep', '.')
@@@ -1392,9 -1593,37 +1594,37 @@@ class ImportConversion(object)
          self.primary_key_indexes = [self.columns.index(col.name) for col in self.table_meta.primary_key]
          self.partition_key_indexes = [self.columns.index(col.name) for col in self.table_meta.partition_key]
  
+         if statement is None:
+             self.use_prepared_statements = False
+             statement = self._get_primary_key_statement(parent, table_meta)
+         else:
+             self.use_prepared_statements = True
+ 
          self.proto_version = statement.protocol_version
-         self.cqltypes = dict([(c.name, c.type) for c in statement.column_metadata])
-         self.converters = dict([(c.name, self._get_converter(c.type)) for c in statement.column_metadata])
+ 
+         # the cql types and converters for the prepared statement, either the full statement or only the primary keys
+         self.cqltypes = [c.type for c in statement.column_metadata]
+         self.converters = [self._get_converter(c.type) for c in statement.column_metadata]
+ 
+         # the cql types for the entire statement, these are the same as the types above but
+         # only when using prepared statements
 -        self.coltypes = [table_meta.columns[name].typestring for name in parent.valid_columns]
++        self.coltypes = [table_meta.columns[name].cql_type for name in parent.valid_columns]
+         # these functions are used for non-prepared statements to protect values with quotes if required
+         self.protectors = [protect_value if t in ('ascii', 'text', 'timestamp', 'date', 'time', 'inet') else lambda v: v
+                            for t in self.coltypes]
+ 
+     @staticmethod
+     def _get_primary_key_statement(parent, table_meta):
+         """
+         We prepare a query statement to find out the types of the partition key columns so we can
+         route the update query to the correct replicas. As far as I understood this is the easiest
+         way to find out the types of the partition columns, we will never use this prepared statement
+         """
+         where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
+         select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(parent.ks),
+                                                          protect_name(parent.table),
+                                                          where_clause)
+         return parent.session.prepare(select_query)
  
      def _get_converter(self, cql_type):
          """
@@@ -1695,67 -2007,88 +2008,88 @@@ class ImportProcess(ChildProcess)
              self._session.cluster.shutdown()
          ChildProcess.close(self)
  
-     def run_counter(self, table_meta):
-         """
-         Main run method for tables that contain counter columns.
-         """
-         query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
- 
-         # We prepare a query statement to find out the types of the partition key columns so we can
-         # route the update query to the correct replicas. As far as I understood this is the easiest
-         # way to find out the types of the partition columns, we will never use this prepared statement
-         where_clause = ' AND '.join(['%s = ?' % (protect_name(c.name)) for c in table_meta.partition_key])
-         select_query = 'SELECT * FROM %s.%s WHERE %s' % (protect_name(self.ks), protect_name(self.table), where_clause)
-         conv = ImportConversion(self, table_meta, self.session.prepare(select_query))
- 
-         while True:
-             batch = self.inmsg.get()
-             try:
-                 for b in self.split_batches(batch, conv):
-                     self.send_counter_batch(query, conv, b)
+     def make_params(self):
+         metadata = self.session.cluster.metadata
+         table_meta = metadata.keyspaces[self.ks].tables[self.table]
+ 
+         prepared_statement = None
 -        is_counter = ("counter" in [table_meta.columns[name].typestring for name in self.valid_columns])
++        is_counter = ("counter" in [table_meta.columns[name].cql_type for name in self.valid_columns])
+         if is_counter:
+             query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
+             make_statement = self.wrap_make_statement(self.make_counter_batch_statement)
+         elif self.use_prepared_statements:
+             query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
+                                                             protect_name(self.table),
+                                                             ', '.join(protect_names(self.valid_columns),),
+                                                             ', '.join(['?' for _ in self.valid_columns]))
+ 
+             query = self.session.prepare(query)
+             query.consistency_level = self.consistency_level
+             prepared_statement = query
+             make_statement = self.wrap_make_statement(self.make_prepared_batch_statement)
+         else:
+             query = 'INSERT INTO %s.%s (%s) VALUES (%%s)' % (protect_name(self.ks),
+                                                              protect_name(self.table),
+                                                              ', '.join(protect_names(self.valid_columns),))
+             make_statement = self.wrap_make_statement(self.make_non_prepared_batch_statement)
  
-             except Exception, exc:
-                 self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                 if self.debug:
-                     traceback.print_exc(exc)
+         conv = ImportConversion(self, table_meta, prepared_statement)
+         tm = TokenMap(self.ks, self.hostname, self.local_dc, self.session)
+         return query, conv, tm, make_statement
  
-     def run_normal(self, table_meta):
+     def inner_run(self, query, conv, tm, make_statement):
          """
-         Main run method for normal tables, i.e. tables that do not contain counter columns.
+         Main run method. Note that we bind self methods that are called inside loops
+         for performance reasons.
          """
-         query = 'INSERT INTO %s.%s (%s) VALUES (%s)' % (protect_name(self.ks),
-                                                         protect_name(self.table),
-                                                         ', '.join(protect_names(self.valid_columns),),
-                                                         ', '.join(['?' for _ in self.valid_columns]))
+         self.query = query
+         self.conv = conv
+         self.make_statement = make_statement
  
-         query_statement = self.session.prepare(query)
-         query_statement.consistency_level = self.consistency_level
-         conv = ImportConversion(self, table_meta, query_statement)
+         convert_rows = self.convert_rows
+         split_into_batches = self.split_into_batches
+         result_callback = self.result_callback
+         err_callback = self.err_callback
+         session = self.session
  
          while True:
-             batch = self.inmsg.get()
+             chunk = self.inmsg.recv()
+             if chunk is None:
+                 break
+ 
              try:
-                 for b in self.split_batches(batch, conv):
-                     self.send_normal_batch(conv, query_statement, b)
+                 chunk['rows'] = convert_rows(conv, chunk)
+                 for replicas, batch in split_into_batches(chunk, conv, tm):
+                     statement = make_statement(query, conv, chunk, batch, replicas)
+                     future = session.execute_async(statement)
+                     future.add_callbacks(callback=result_callback, callback_args=(batch, chunk),
+                                          errback=err_callback, errback_args=(batch, chunk, replicas))
  
              except Exception, exc:
-                 self.outmsg.put((batch, '%s - %s' % (exc.__class__.__name__, exc.message)))
-                 if self.debug:
-                     traceback.print_exc(exc)
+                 self.report_error(exc, chunk, chunk['rows'])
  
-     def send_counter_batch(self, query_text, conv, batch):
-         if self.test_failures and self.maybe_inject_failures(batch):
-             return
+     def wrap_make_statement(self, inner_make_statement):
+         def make_statement(query, conv, chunk, batch, replicas):
+             try:
+                 return inner_make_statement(query, conv, batch, replicas)
+             except Exception, exc:
+                 print "Failed to make batch statement: {}".format(exc)
+                 self.report_error(exc, chunk, batch['rows'])
+                 return None
  
-         error_rows = []
-         batch_statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+         def make_statement_with_failures(query, conv, chunk, batch, replicas):
+             failed_batch = self.maybe_inject_failures(batch)
+             if failed_batch:
+                 return failed_batch
+             return make_statement(query, conv, chunk, batch, replicas)
  
-         for r in batch['rows']:
-             row = self.filter_row_values(r)
-             if len(row) != len(self.valid_columns):
-                 error_rows.append(row)
-                 continue
+         return make_statement_with_failures if self.test_failures else make_statement
  
+     def make_counter_batch_statement(self, query, conv, batch, replicas):
+         statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+         statement.replicas = replicas
+         statement.keyspace = self.ks
+         for row in batch['rows']:
              where_clause = []
              set_clause = []
              for i, value in enumerate(row):


[11/23] cassandra git commit: Merge commit 'c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc' into cassandra-2.2

Posted by sl...@apache.org.
Merge commit 'c3d2f26f46c2d37b6cf918cbb5565fe57a5904cc' into cassandra-2.2


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b74ffeaf
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b74ffeaf
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b74ffeaf

Branch: refs/heads/cassandra-2.2
Commit: b74ffeafd2e08a7669013d6b4ba87980e1ad5379
Parents: 6e0395e c3d2f26
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue Mar 8 10:22:22 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Mar 8 10:23:08 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                |    1 +
 bin/cqlsh.py               |   28 +-
 pylib/cqlshlib/copyutil.py | 1160 +++++++++++++++++++++++++--------------
 pylib/cqlshlib/util.py     |   35 +-
 pylib/setup.py             |    2 +
 5 files changed, 809 insertions(+), 417 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b74ffeaf/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 89fa904,d6b085c..7b67cdc
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,30 -1,5 +1,31 @@@
 -2.1.14
 +2.2.6
 + * Fix AE in nodetool cfstats (backport CASSANDRA-10859) (CASSANDRA-11297)
 + * Unresolved hostname leads to replace being ignored (CASSANDRA-11210)
 + * Fix filtering on non-primary key columns for thrift static column families
 +   (CASSANDRA-6377)
 + * Only log yaml config once, at startup (CASSANDRA-11217)
 + * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
 + * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
 + * Range.compareTo() violates the contract of Comparable (CASSANDRA-11216)
 + * Avoid NPE when serializing ErrorMessage with null message (CASSANDRA-11167)
 + * Replacing an aggregate with a new version doesn't reset INITCOND (CASSANDRA-10840)
 + * (cqlsh) cqlsh cannot be called through symlink (CASSANDRA-11037)
 + * fix ohc and java-driver pom dependencies in build.xml (CASSANDRA-10793)
 + * Protect from keyspace dropped during repair (CASSANDRA-11065)
 + * Handle adding fields to a UDT in SELECT JSON and toJson() (CASSANDRA-11146)
 + * Better error message for cleanup (CASSANDRA-10991)
 + * cqlsh pg-style-strings broken if line ends with ';' (CASSANDRA-11123)
 + * Use cloned TokenMetadata in size estimates to avoid race against membership check
 +   (CASSANDRA-10736)
 + * Always persist upsampled index summaries (CASSANDRA-10512)
 + * (cqlsh) Fix inconsistent auto-complete (CASSANDRA-10733)
 + * Make SELECT JSON and toJson() threadsafe (CASSANDRA-11048)
 + * Fix SELECT on tuple relations for mixed ASC/DESC clustering order (CASSANDRA-7281)
 + * (cqlsh) Support utf-8/cp65001 encoding on Windows (CASSANDRA-11030)
 + * Fix paging on DISTINCT queries repeats result when first row in partition changes
 +   (CASSANDRA-10010)
 +Merged from 2.1:
+  * COPY FROM on large datasets: fix progress report and debug performance (CASSANDRA-11053)
   * InvalidateKeys should have a weak ref to key cache (CASSANDRA-11176)
   * Don't remove FailureDetector history on removeEndpoint (CASSANDRA-10371)
   * Only notify if repair status changed (CASSANDRA-11172)