You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2014/12/09 14:00:10 UTC

[1/3] cassandra git commit: Throw correct exception when trying to bind a keyspace or table name

Repository: cassandra
Updated Branches:
  refs/heads/trunk 7d9acdb10 -> 9510eb0f3


Throw correct exception when trying to bind a keyspace or table name

patch by Benjamin Lerer; reviewed by Aleksey Yeschenko for
CASSANDRA-6952


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

Branch: refs/heads/trunk
Commit: 71e1219de71fe341316f8cab2f5faeea37ceff1e
Parents: c26a7e7
Author: Benjamin Lerer <b_...@hotmail.com>
Authored: Tue Dec 9 13:56:25 2014 +0100
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Tue Dec 9 13:56:25 2014 +0100

----------------------------------------------------------------------
 CHANGES.txt                              | 2 ++
 src/java/org/apache/cassandra/cql3/Cql.g | 1 +
 2 files changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/71e1219d/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5573316..7c89e60 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,6 @@
 2.0.12:
+ * Throw correct exception when trying to bind a keyspace or table
+   name (CASSANDRA-6952)
  * Make HHOM.compact synchronized (CASSANDRA-8416)
  * cancel latency-sampling task when CF is dropped (CASSANDRA-8401)
  * don't block SocketThread for MessagingService (CASSANDRA-8188)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/71e1219d/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index 43b1c01..a80746c 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -771,6 +771,7 @@ cfOrKsName[CFName name, boolean isKs]
     : t=IDENT              { if (isKs) $name.setKeyspace($t.text, false); else $name.setColumnFamily($t.text, false); }
     | t=QUOTED_NAME        { if (isKs) $name.setKeyspace($t.text, true); else $name.setColumnFamily($t.text, true); }
     | k=unreserved_keyword { if (isKs) $name.setKeyspace(k, false); else $name.setColumnFamily(k, false); }
+    | QMARK {addRecognitionError("Bind variables cannot be used for keyspace or table names");}
     ;
 
 constant returns [Constants.Literal constant]


[3/3] cassandra git commit: Merge branch 'cassandra-2.1' into trunk

Posted by al...@apache.org.
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 9510eb0f3cf9f3cb085c218242e31627c37383b9
Parents: 7d9acdb 7fa0341
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Tue Dec 9 13:59:56 2014 +0100
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Tue Dec 9 13:59:56 2014 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 ++
 src/java/org/apache/cassandra/cql3/Cql.g        |  1 +
 .../org/apache/cassandra/cql3/CQLTester.java    | 25 +++++++++++++++--
 .../apache/cassandra/cql3/UseStatementTest.java | 29 ++++++++++++++++++++
 4 files changed, 54 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9510eb0f/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9510eb0f/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------


[2/3] cassandra git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by al...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1

Conflicts:
	CHANGES.txt


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

Branch: refs/heads/trunk
Commit: 7fa034136a96aab4a8ca5173751bdb52bcbca500
Parents: c1610f6 71e1219
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Tue Dec 9 13:59:18 2014 +0100
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Tue Dec 9 13:59:18 2014 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 ++
 src/java/org/apache/cassandra/cql3/Cql.g        |  1 +
 .../org/apache/cassandra/cql3/CQLTester.java    | 25 +++++++++++++++--
 .../apache/cassandra/cql3/UseStatementTest.java | 29 ++++++++++++++++++++
 4 files changed, 54 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7fa03413/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index b8d2ff6,7c89e60..28dd6b4
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,23 -1,6 +1,25 @@@
 -2.0.12:
 +2.1.3
 + * Reduce maxHintsInProgress (CASSANDRA-8415)
 + * BTree updates may call provided update function twice (CASSANDRA-8018)
 + * Release sstable references after anticompaction (CASSANDRA-8386)
 + * Handle abort() in SSTableRewriter properly (CASSANDRA-8320)
 + * Fix high size calculations for prepared statements (CASSANDRA-8231)
 + * Centralize shared executors (CASSANDRA-8055)
 + * Fix filtering for CONTAINS (KEY) relations on frozen collection
 +   clustering columns when the query is restricted to a single
 +   partition (CASSANDRA-8203)
 + * Do more aggressive entire-sstable TTL expiry checks (CASSANDRA-8243)
 + * Add more log info if readMeter is null (CASSANDRA-8238)
 + * add check of the system wall clock time at startup (CASSANDRA-8305)
 + * Support for frozen collections (CASSANDRA-7859)
 + * Fix overflow on histogram computation (CASSANDRA-8028)
 + * Have paxos reuse the timestamp generation of normal queries (CASSANDRA-7801)
 + * Fix incremental repair not remove parent session on remote (CASSANDRA-8291)
 + * Improve JBOD disk utilization (CASSANDRA-7386)
 + * Log failed host when preparing incremental repair (CASSANDRA-8228)
 +Merged from 2.0:
+  * Throw correct exception when trying to bind a keyspace or table
+    name (CASSANDRA-6952)
   * Make HHOM.compact synchronized (CASSANDRA-8416)
   * cancel latency-sampling task when CF is dropped (CASSANDRA-8401)
   * don't block SocketThread for MessagingService (CASSANDRA-8188)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7fa03413/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7fa03413/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/CQLTester.java
index dd22896,0000000..2226153
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@@ -1,756 -1,0 +1,775 @@@
 +/*
 + * 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.
 + */
 +package org.apache.cassandra.cql3;
 +
 +import java.io.File;
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +import java.util.concurrent.CountDownLatch;
 +import java.util.concurrent.ExecutionException;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import com.google.common.base.Objects;
 +import com.google.common.collect.ImmutableSet;
 +import org.junit.AfterClass;
 +import org.junit.After;
 +import org.junit.Assert;
 +import org.junit.BeforeClass;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.SchemaLoader;
 +import org.apache.cassandra.concurrent.ScheduledExecutors;
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.Schema;
 +import org.apache.cassandra.db.Directories;
 +import org.apache.cassandra.db.Keyspace;
 +import org.apache.cassandra.db.marshal.*;
 +import org.apache.cassandra.exceptions.*;
 +import org.apache.cassandra.io.util.FileUtils;
 +import org.apache.cassandra.serializers.TypeSerializer;
 +
 +/**
 + * Base class for CQL tests.
 + */
 +public abstract class CQLTester
 +{
 +    protected static final Logger logger = LoggerFactory.getLogger(CQLTester.class);
 +
 +    public static final String KEYSPACE = "cql_test_keyspace";
 +    private static final boolean USE_PREPARED_VALUES = Boolean.valueOf(System.getProperty("cassandra.test.use_prepared", "true"));
 +    private static final AtomicInteger seqNumber = new AtomicInteger();
 +
 +    static
 +    {
 +        // Once per-JVM is enough
 +        SchemaLoader.prepareServer();
 +    }
 +
 +    private String currentTable;
 +    private final Set<String> currentTypes = new HashSet<>();
 +
 +    @BeforeClass
 +    public static void setUpClass() throws Throwable
 +    {
 +        schemaChange(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE));
 +    }
 +
 +    @AfterClass
 +    public static void tearDownClass()
 +    {
 +    }
 +
 +    @After
 +    public void afterTest() throws Throwable
 +    {
 +        if (currentTable == null)
 +            return;
 +
 +        final String tableToDrop = currentTable;
 +        final Set<String> typesToDrop = currentTypes.isEmpty() ? Collections.emptySet() : new HashSet(currentTypes);
 +        currentTable = null;
 +        currentTypes.clear();
 +
 +        // We want to clean up after the test, but dropping a table is rather long so just do that asynchronously
 +        ScheduledExecutors.optionalTasks.execute(new Runnable()
 +        {
 +            public void run()
 +            {
 +                try
 +                {
 +                    schemaChange(String.format("DROP TABLE IF EXISTS %s.%s", KEYSPACE, tableToDrop));
 +
 +                    for (String typeName : typesToDrop)
 +                        schemaChange(String.format("DROP TYPE IF EXISTS %s.%s", KEYSPACE, typeName));
 +
 +                    // Dropping doesn't delete the sstables. It's not a huge deal but it's cleaner to cleanup after us
 +                    // Thas said, we shouldn't delete blindly before the SSTableDeletingTask for the table we drop
 +                    // have run or they will be unhappy. Since those taks are scheduled on StorageService.tasks and that's
 +                    // mono-threaded, just push a task on the queue to find when it's empty. No perfect but good enough.
 +
 +                    final CountDownLatch latch = new CountDownLatch(1);
 +                    ScheduledExecutors.nonPeriodicTasks.execute(new Runnable()
 +                    {
 +                            public void run()
 +                            {
 +                                latch.countDown();
 +                            }
 +                    });
 +                    latch.await(2, TimeUnit.SECONDS);
 +
 +                    removeAllSSTables(KEYSPACE, tableToDrop);
 +                }
 +                catch (Exception e)
 +                {
 +                    throw new RuntimeException(e);
 +                }
 +            }
 +        });
 +    }
 +
 +    public void flush()
 +    {
 +        try
 +        {
 +            if (currentTable != null)
 +                Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable).forceFlush().get();
 +        }
 +        catch (InterruptedException e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +        catch (ExecutionException e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +    }
 +
 +    public boolean usePrepared()
 +    {
 +        return USE_PREPARED_VALUES;
 +    }
 +
 +    private static void removeAllSSTables(String ks, String table)
 +    {
 +        // clean up data directory which are stored as data directory/keyspace/data files
 +        for (File d : Directories.getKSChildDirectories(ks))
 +        {
 +            if (d.exists() && d.getName().contains(table))
 +                FileUtils.deleteRecursive(d);
 +        }
 +    }
 +
 +    protected String keyspace()
 +    {
 +        return KEYSPACE;
 +    }
 +
 +    protected String currentTable()
 +    {
 +        return currentTable;
 +    }
 +
 +    protected String createType(String query)
 +    {
 +        String typeName = "type_" + seqNumber.getAndIncrement();
 +        String fullQuery = String.format(query, KEYSPACE + "." + typeName);
 +        currentTypes.add(typeName);
 +        logger.info(fullQuery);
 +        schemaChange(fullQuery);
 +        return typeName;
 +    }
 +
 +    protected void createTable(String query)
 +    {
 +        currentTable = "table_" + seqNumber.getAndIncrement();
 +        String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
 +        logger.info(fullQuery);
 +        schemaChange(fullQuery);
 +    }
 +
 +    protected void createTableMayThrow(String query) throws Throwable
 +    {
 +        currentTable = "table_" + seqNumber.getAndIncrement();
 +        String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
 +        logger.info(fullQuery);
 +        try
 +        {
 +            QueryProcessor.executeOnceInternal(fullQuery);
 +        }
 +        catch (RuntimeException ex)
 +        {
 +            throw ex.getCause();
 +        }
 +    }
 +
 +    protected void alterTable(String query)
 +    {
 +        String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
 +        logger.info(fullQuery);
 +        schemaChange(fullQuery);
 +    }
 +
 +    protected void alterTableMayThrow(String query) throws Throwable
 +    {
 +        String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
 +        logger.info(fullQuery);
 +        try
 +        {
 +            QueryProcessor.executeOnceInternal(fullQuery);
 +        }
 +        catch (RuntimeException ex)
 +        {
 +            throw ex.getCause();
 +        }
 +    }
 +
 +    protected void dropTable(String query)
 +    {
 +        String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
 +        logger.info(fullQuery);
 +        schemaChange(fullQuery);
 +    }
 +
 +    protected void createIndex(String query)
 +    {
 +        String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
 +        logger.info(fullQuery);
 +        schemaChange(fullQuery);
 +    }
 +
 +    protected void createIndexMayThrow(String query) throws Throwable
 +    {
 +        String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
 +        logger.info(fullQuery);
 +        try
 +        {
 +            QueryProcessor.executeOnceInternal(fullQuery);
 +        }
 +        catch (RuntimeException ex)
 +        {
 +            throw ex.getCause();
 +        }
 +    }
 +
 +    protected void dropIndex(String query) throws Throwable
 +    {
 +        String fullQuery = String.format(query, KEYSPACE);
 +        logger.info(fullQuery);
 +        schemaChange(fullQuery);
 +    }
 +
 +    private static void schemaChange(String query)
 +    {
 +        try
 +        {
 +            // executeOnceInternal don't work for schema changes
 +            QueryProcessor.executeOnceInternal(query);
 +        }
 +        catch (Exception e)
 +        {
 +            throw new RuntimeException("Error setting schema for test (query was: " + query + ")", e);
 +        }
 +    }
 +
 +    protected CFMetaData currentTableMetadata()
 +    {
 +        return Schema.instance.getCFMetaData(KEYSPACE, currentTable);
 +    }
 +
 +    protected UntypedResultSet execute(String query, Object... values) throws Throwable
 +    {
 +        try
 +        {
 +            query = currentTable == null ? query : String.format(query, KEYSPACE + "." + currentTable);
 +
 +            UntypedResultSet rs;
 +            if (USE_PREPARED_VALUES)
 +            {
 +                logger.info("Executing: {} with values {}", query, formatAllValues(values));
 +                rs = QueryProcessor.executeOnceInternal(query, transformValues(values));
 +            }
 +            else
 +            {
 +                query = replaceValues(query, values);
 +                logger.info("Executing: {}", query);
 +                rs = QueryProcessor.executeOnceInternal(query);
 +            }
 +            if (rs != null)
 +                logger.info("Got {} rows", rs.size());
 +            return rs;
 +        }
 +        catch (RuntimeException e)
 +        {
 +            Throwable cause = e.getCause() != null ? e.getCause() : e;
 +            logger.info("Got error: {}", cause.getMessage() == null ? cause.toString() : cause.getMessage());
 +            throw cause;
 +        }
 +    }
 +
 +    protected void assertRows(UntypedResultSet result, Object[]... rows)
 +    {
 +        if (result == null)
 +        {
 +            if (rows.length > 0)
 +                Assert.fail(String.format("No rows returned by query but %d expected", rows.length));
 +            return;
 +        }
 +
 +        List<ColumnSpecification> meta = result.metadata();
 +        Iterator<UntypedResultSet.Row> iter = result.iterator();
 +        int i = 0;
 +        while (iter.hasNext() && i < rows.length)
 +        {
 +            Object[] expected = rows[i];
 +            UntypedResultSet.Row actual = iter.next();
 +
 +            Assert.assertEquals(String.format("Invalid number of (expected) values provided for row %d", i), meta.size(), expected.length);
 +
 +            for (int j = 0; j < meta.size(); j++)
 +            {
 +                ColumnSpecification column = meta.get(j);
 +                Object expectedValue = expected[j];
 +                ByteBuffer expectedByteValue = makeByteBuffer(expected[j], (AbstractType)column.type);
 +                ByteBuffer actualValue = actual.getBytes(column.name.toString());
 +
 +                if (!Objects.equal(expectedByteValue, actualValue))
 +                    Assert.fail(String.format("Invalid value for row %d column %d (%s of type %s), expected <%s> but got <%s>",
 +                                              i, j, column.name, column.type.asCQL3Type(), formatValue(expectedByteValue, column.type), formatValue(actualValue, column.type)));
 +            }
 +            i++;
 +        }
 +
 +        if (iter.hasNext())
 +        {
 +            while (iter.hasNext())
 +            {
 +                iter.next();
 +                i++;
 +            }
 +            Assert.fail(String.format("Got less rows than expected. Expected %d but got %d.", rows.length, i));
 +        }
 +
 +        Assert.assertTrue(String.format("Got more rows than expected. Expected %d but got %d", rows.length, i), i == rows.length);
 +    }
 +
 +    protected void assertAllRows(Object[]... rows) throws Throwable
 +    {
 +        assertRows(execute("SELECT * FROM %s"), rows);
 +    }
 +
 +    protected Object[] row(Object... expected)
 +    {
 +        return expected;
 +    }
 +
 +    protected void assertEmpty(UntypedResultSet result) throws Throwable
 +    {
 +        if (result != null && result.size() != 0)
 +            throw new InvalidRequestException(String.format("Expected empty result but got %d rows", result.size()));
 +    }
 +
 +    protected void assertInvalid(String query, Object... values) throws Throwable
 +    {
 +        assertInvalidMessage(null, query, values);
 +    }
 +
 +    protected void assertInvalidMessage(String errorMessage, String query, Object... values) throws Throwable
 +    {
 +        try
 +        {
 +            execute(query, values);
 +            String q = USE_PREPARED_VALUES
 +                     ? query + " (values: " + formatAllValues(values) + ")"
 +                     : replaceValues(query, values);
 +            Assert.fail("Query should be invalid but no error was thrown. Query is: " + q);
 +        }
 +        catch (InvalidRequestException e)
 +        {
 +            if (errorMessage != null)
 +            {
-                 Assert.assertTrue("Expected error message to contain '" + errorMessage + "', but got '" + e.getMessage() + "'",
-                         e.getMessage().contains(errorMessage));
++                assertMessageContains(errorMessage, e);
 +            }
 +        }
 +    }
 +
 +    protected void assertInvalidSyntax(String query, Object... values) throws Throwable
 +    {
++        assertInvalidSyntaxMessage(null, query, values);
++    }
++
++    protected void assertInvalidSyntaxMessage(String errorMessage, String query, Object... values) throws Throwable
++    {
 +        try
 +        {
 +            execute(query, values);
 +            String q = USE_PREPARED_VALUES
 +                     ? query + " (values: " + formatAllValues(values) + ")"
 +                     : replaceValues(query, values);
 +            Assert.fail("Query should have invalid syntax but no error was thrown. Query is: " + q);
 +        }
 +        catch (SyntaxException e)
 +        {
-             // This is what we expect
++            if (errorMessage != null)
++            {
++                assertMessageContains(errorMessage, e);
++            }
 +        }
 +    }
 +
++    /**
++     * Asserts that the message of the specified exception contains the specified text.
++     *
++     * @param text the text that the exception message must contains
++     * @param e the exception to check
++     */
++    private static void assertMessageContains(String text, Exception e)
++    {
++        Assert.assertTrue("Expected error message to contain '" + text + "', but got '" + e.getMessage() + "'",
++                e.getMessage().contains(text));
++    }
++
 +    private static String replaceValues(String query, Object[] values)
 +    {
 +        StringBuilder sb = new StringBuilder();
 +        int last = 0;
 +        int i = 0;
 +        int idx;
 +        while ((idx = query.indexOf('?', last)) > 0)
 +        {
 +            if (i >= values.length)
 +                throw new IllegalArgumentException(String.format("Not enough values provided. The query has at least %d variables but only %d values provided", i, values.length));
 +
 +            sb.append(query.substring(last, idx));
 +
 +            Object value = values[i++];
 +
 +            // When we have a .. IN ? .., we use a list for the value because that's what's expected when the value is serialized.
 +            // When we format as string however, we need to special case to use parenthesis. Hackish but convenient.
 +            if (idx >= 3 && value instanceof List && query.substring(idx - 3, idx).equalsIgnoreCase("IN "))
 +            {
 +                List l = (List)value;
 +                sb.append("(");
 +                for (int j = 0; j < l.size(); j++)
 +                {
 +                    if (j > 0)
 +                        sb.append(", ");
 +                    sb.append(formatForCQL(l.get(j)));
 +                }
 +                sb.append(")");
 +            }
 +            else
 +            {
 +                sb.append(formatForCQL(value));
 +            }
 +            last = idx + 1;
 +        }
 +        sb.append(query.substring(last));
 +        return sb.toString();
 +    }
 +
 +    // We're rellly only returning ByteBuffers but this make the type system happy
 +    private static Object[] transformValues(Object[] values)
 +    {
 +        // We could partly rely on QueryProcessor.executeOnceInternal doing type conversion for us, but
 +        // it would complain with ClassCastException if we pass say a string where an int is excepted (since
 +        // it bases conversion on what the value should be, not what it is). For testing, we sometimes
 +        // want to pass value of the wrong type and assert that this properly raise an InvalidRequestException
 +        // and executeOnceInternal goes into way. So instead, we pre-convert everything to bytes here based
 +        // on the value.
 +        // Besides, we need to handle things like TupleValue that executeOnceInternal don't know about.
 +
 +        Object[] buffers = new ByteBuffer[values.length];
 +        for (int i = 0; i < values.length; i++)
 +        {
 +            Object value = values[i];
 +            if (value == null)
 +            {
 +                buffers[i] = null;
 +                continue;
 +            }
 +
 +            try
 +            {
 +                buffers[i] = typeFor(value).decompose(serializeTuples(value));
 +            }
 +            catch (Exception ex)
 +            {
 +                logger.info("Error serializing query parameter {}:", value, ex);
 +                throw ex;
 +            }
 +        }
 +        return buffers;
 +    }
 +
 +    private static Object serializeTuples(Object value)
 +    {
 +        if (value instanceof TupleValue)
 +        {
 +            return ((TupleValue)value).toByteBuffer();
 +        }
 +
 +        // We need to reach inside collections for TupleValue and transform them to ByteBuffer
 +        // since otherwise the decompose method of the collection AbstractType won't know what
 +        // to do with them
 +        if (value instanceof List)
 +        {
 +            List l = (List)value;
 +            List n = new ArrayList(l.size());
 +            for (Object o : l)
 +                n.add(serializeTuples(o));
 +            return n;
 +        }
 +
 +        if (value instanceof Set)
 +        {
 +            Set s = (Set)value;
 +            Set n = new LinkedHashSet(s.size());
 +            for (Object o : s)
 +                n.add(serializeTuples(o));
 +            return n;
 +        }
 +
 +        if (value instanceof Map)
 +        {
 +            Map m = (Map)value;
 +            Map n = new LinkedHashMap(m.size());
 +            for (Object entry : m.entrySet())
 +                n.put(serializeTuples(((Map.Entry)entry).getKey()), serializeTuples(((Map.Entry)entry).getValue()));
 +            return n;
 +        }
 +        return value;
 +    }
 +
 +    private static String formatAllValues(Object[] values)
 +    {
 +        StringBuilder sb = new StringBuilder();
 +        sb.append("[");
 +        for (int i = 0; i < values.length; i++)
 +        {
 +            if (i > 0)
 +                sb.append(", ");
 +            sb.append(formatForCQL(values[i]));
 +        }
 +        sb.append("]");
 +        return sb.toString();
 +    }
 +
 +    private static String formatForCQL(Object value)
 +    {
 +        if (value == null)
 +            return "null";
 +
 +        if (value instanceof TupleValue)
 +            return ((TupleValue)value).toCQLString();
 +
 +        // We need to reach inside collections for TupleValue. Besides, for some reason the format
 +        // of collection that CollectionType.getString gives us is not at all 'CQL compatible'
 +        if (value instanceof Collection || value instanceof Map)
 +        {
 +            StringBuilder sb = new StringBuilder();
 +            if (value instanceof List)
 +            {
 +                List l = (List)value;
 +                sb.append("[");
 +                for (int i = 0; i < l.size(); i++)
 +                {
 +                    if (i > 0)
 +                        sb.append(", ");
 +                    sb.append(formatForCQL(l.get(i)));
 +                }
 +                sb.append("]");
 +            }
 +            else if (value instanceof Set)
 +            {
 +                Set s = (Set)value;
 +                sb.append("{");
 +                Iterator iter = s.iterator();
 +                while (iter.hasNext())
 +                {
 +                    sb.append(formatForCQL(iter.next()));
 +                    if (iter.hasNext())
 +                        sb.append(", ");
 +                }
 +                sb.append("}");
 +            }
 +            else
 +            {
 +                Map m = (Map)value;
 +                sb.append("{");
 +                Iterator iter = m.entrySet().iterator();
 +                while (iter.hasNext())
 +                {
 +                    Map.Entry entry = (Map.Entry)iter.next();
 +                    sb.append(formatForCQL(entry.getKey())).append(": ").append(formatForCQL(entry.getValue()));
 +                    if (iter.hasNext())
 +                        sb.append(", ");
 +                }
 +                sb.append("}");
 +            }
 +            return sb.toString();
 +        }
 +
 +        AbstractType type = typeFor(value);
 +        String s = type.getString(type.decompose(value));
 +
 +        if (type instanceof UTF8Type)
 +            return String.format("'%s'", s.replaceAll("'", "''"));
 +
 +        if (type instanceof BytesType)
 +            return "0x" + s;
 +
 +        return s;
 +    }
 +
 +    private static ByteBuffer makeByteBuffer(Object value, AbstractType type)
 +    {
 +        if (value == null)
 +            return null;
 +
 +        if (value instanceof TupleValue)
 +            return ((TupleValue)value).toByteBuffer();
 +
 +        if (value instanceof ByteBuffer)
 +            return (ByteBuffer)value;
 +
 +        return type.decompose(value);
 +    }
 +
 +    private static String formatValue(ByteBuffer bb, AbstractType<?> type)
 +    {
 +        if (bb == null)
 +            return "null";
 +
 +        if (type instanceof CollectionType)
 +        {
 +            // CollectionType override getString() to use hexToBytes. We can't change that
 +            // without breaking SSTable2json, but the serializer for collection have the
 +            // right getString so using it directly instead.
 +            TypeSerializer ser = type.getSerializer();
 +            return ser.toString(ser.deserialize(bb));
 +        }
 +
 +        return type.getString(bb);
 +    }
 +
 +    protected Object tuple(Object...values)
 +    {
 +        return new TupleValue(values);
 +    }
 +
 +    protected Object list(Object...values)
 +    {
 +        return Arrays.asList(values);
 +    }
 +
 +    protected Object set(Object...values)
 +    {
 +        return ImmutableSet.copyOf(values);
 +    }
 +
 +    protected Object map(Object...values)
 +    {
 +        if (values.length % 2 != 0)
 +            throw new IllegalArgumentException();
 +
 +        int size = values.length / 2;
 +        Map m = new LinkedHashMap(size);
 +        for (int i = 0; i < size; i++)
 +            m.put(values[2 * i], values[(2 * i) + 1]);
 +        return m;
 +    }
 +
 +    // Attempt to find an AbstracType from a value (for serialization/printing sake).
 +    // Will work as long as we use types we know of, which is good enough for testing
 +    private static AbstractType typeFor(Object value)
 +    {
 +        if (value instanceof ByteBuffer || value instanceof TupleValue || value == null)
 +            return BytesType.instance;
 +
 +        if (value instanceof Integer)
 +            return Int32Type.instance;
 +
 +        if (value instanceof Long)
 +            return LongType.instance;
 +
 +        if (value instanceof Float)
 +            return FloatType.instance;
 +
 +        if (value instanceof Double)
 +            return DoubleType.instance;
 +
 +        if (value instanceof String)
 +            return UTF8Type.instance;
 +
 +        if (value instanceof Boolean)
 +            return BooleanType.instance;
 +
 +        if (value instanceof List)
 +        {
 +            List l = (List)value;
 +            AbstractType elt = l.isEmpty() ? BytesType.instance : typeFor(l.get(0));
 +            return ListType.getInstance(elt, true);
 +        }
 +
 +        if (value instanceof Set)
 +        {
 +            Set s = (Set)value;
 +            AbstractType elt = s.isEmpty() ? BytesType.instance : typeFor(s.iterator().next());
 +            return SetType.getInstance(elt, true);
 +        }
 +
 +        if (value instanceof Map)
 +        {
 +            Map m = (Map)value;
 +            AbstractType keys, values;
 +            if (m.isEmpty())
 +            {
 +                keys = BytesType.instance;
 +                values = BytesType.instance;
 +            }
 +            else
 +            {
 +                Map.Entry entry = (Map.Entry)m.entrySet().iterator().next();
 +                keys = typeFor(entry.getKey());
 +                values = typeFor(entry.getValue());
 +            }
 +            return MapType.getInstance(keys, values, true);
 +        }
 +
 +        throw new IllegalArgumentException("Unsupported value type (value is " + value + ")");
 +    }
 +
 +    private static class TupleValue
 +    {
 +        private final Object[] values;
 +
 +        TupleValue(Object[] values)
 +        {
 +            this.values = values;
 +        }
 +
 +        public ByteBuffer toByteBuffer()
 +        {
 +            ByteBuffer[] bbs = new ByteBuffer[values.length];
 +            for (int i = 0; i < values.length; i++)
 +                bbs[i] = makeByteBuffer(values[i], typeFor(values[i]));
 +            return TupleType.buildValue(bbs);
 +        }
 +
 +        public String toCQLString()
 +        {
 +            StringBuilder sb = new StringBuilder();
 +            sb.append("(");
 +            for (int i = 0; i < values.length; i++)
 +            {
 +                if (i > 0)
 +                    sb.append(", ");
 +                sb.append(formatForCQL(values[i]));
 +            }
 +            sb.append(")");
 +            return sb.toString();
 +        }
 +
 +        public String toString()
 +        {
 +            return "TupleValue" + toCQLString();
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7fa03413/test/unit/org/apache/cassandra/cql3/UseStatementTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/UseStatementTest.java
index 0000000,0000000..77ac8a7
new file mode 100644
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/UseStatementTest.java
@@@ -1,0 -1,0 +1,29 @@@
++/*
++ * 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.
++ */
++package org.apache.cassandra.cql3;
++
++import org.junit.Test;
++
++public class UseStatementTest extends CQLTester
++{
++    @Test
++    public void testUseStatementWithBindVariable() throws Throwable
++    {
++        assertInvalidSyntaxMessage("Bind variables cannot be used for keyspace or table names", "USE ?");
++    }
++}