You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ty...@apache.org on 2014/08/20 18:13:56 UTC

[1/2] git commit: Fix ReversedType.isCompatibleWith()

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1.0 45159ae7b -> d3450570c


Fix ReversedType.isCompatibleWith()

Patch by Tyler Hobbs; reviewed by Sylvain Lebresne for CASSANDRA-7797


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

Branch: refs/heads/cassandra-2.1.0
Commit: a4e108c40b234fa754a56a21b63635b83f90aceb
Parents: 4fc417c
Author: Tyler Hobbs <ty...@datastax.com>
Authored: Wed Aug 20 10:58:54 2014 -0500
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Wed Aug 20 11:01:28 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 ++
 .../cassandra/db/marshal/ReversedType.java      |  9 ++++++
 .../org/apache/cassandra/cql3/TypeTest.java     | 34 ++++++++++++++++++++
 3 files changed, 45 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e108c4/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index fe9f4e0..94bdd89 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,6 @@
 2.0.10
+ * Fix ALTER clustering column type from DateType to TimestampType when
+   using DESC clustering order (CASSANRDA-7797)
  * Stop inheriting liveRatio and liveRatioComputedAt from previous
    memtables (CASSANDRA-7796)
  * Throw EOFException if we run out of chunks in compressed datafile

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e108c4/src/java/org/apache/cassandra/db/marshal/ReversedType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/ReversedType.java b/src/java/org/apache/cassandra/db/marshal/ReversedType.java
index cd61bbe..ffb0229 100644
--- a/src/java/org/apache/cassandra/db/marshal/ReversedType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ReversedType.java
@@ -84,6 +84,15 @@ public class ReversedType<T> extends AbstractType<T>
     }
 
     @Override
+    public boolean isCompatibleWith(AbstractType<?> otherType)
+    {
+        if (!(otherType instanceof ReversedType))
+            return false;
+
+        return this.baseType.isCompatibleWith(((ReversedType) otherType).baseType);
+    }
+
+    @Override
     public boolean isValueCompatibleWith(AbstractType<?> otherType)
     {
         return this.baseType.isValueCompatibleWith(otherType);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4e108c4/test/unit/org/apache/cassandra/cql3/TypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/TypeTest.java b/test/unit/org/apache/cassandra/cql3/TypeTest.java
index f911a44..b08ca2c 100644
--- a/test/unit/org/apache/cassandra/cql3/TypeTest.java
+++ b/test/unit/org/apache/cassandra/cql3/TypeTest.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.cql3;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.gms.Gossiper;
@@ -35,6 +36,7 @@ import org.slf4j.LoggerFactory;
 import static org.apache.cassandra.cql3.QueryProcessor.process;
 import static org.apache.cassandra.cql3.QueryProcessor.processInternal;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 public class TypeTest
 {
@@ -141,4 +143,36 @@ public class TypeTest
         results = executePrepared(prepare(select), QueryOptions.DEFAULT);
         assertEquals(2, results.size());
     }
+
+    @Test
+    // tests CASSANDRA-7797
+    public void testAlterReversedColumn() throws Throwable
+    {
+        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.test_alter_reversed (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
+        executeSchemaChange("ALTER TABLE %s.test_alter_reversed ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.TimestampType)'");
+    }
+
+    @Test
+    public void testIncompatibleReversedTypes() throws Throwable
+    {
+        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.test_incompatible_reversed (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
+        try
+        {
+            executeSchemaChange("ALTER TABLE %s.test_incompatible_reversed ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.TimeUUIDType)'");
+            fail("Expected error for ALTER statement");
+        }
+        catch (ConfigurationException e) { }
+    }
+
+    @Test
+    public void testReversedAndNonReversed() throws Throwable
+    {
+        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.test_reversed_and_non_reversed (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b))");
+        try
+        {
+            executeSchemaChange("ALTER TABLE %s.test_reversed_and_non_reversed ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.DateType)'");
+            fail("Expected error for ALTER statement");
+        }
+        catch (ConfigurationException e) { }
+    }
 }
\ No newline at end of file


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

Posted by ty...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1.0

Conflicts:
	CHANGES.txt
	test/unit/org/apache/cassandra/cql3/TypeTest.java


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

Branch: refs/heads/cassandra-2.1.0
Commit: d3450570ca89ce358355003ef5476dec3fad3e91
Parents: 45159ae a4e108c
Author: Tyler Hobbs <ty...@datastax.com>
Authored: Wed Aug 20 11:13:44 2014 -0500
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Wed Aug 20 11:13:44 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  4 ++-
 .../cassandra/db/marshal/ReversedType.java      |  9 +++++
 .../org/apache/cassandra/cql3/CQLTester.java    |  7 ++++
 .../org/apache/cassandra/cql3/TypeTest.java     | 37 +++++++++++++++++++-
 4 files changed, 55 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d3450570/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index ebd74b2,94bdd89..8fac3b1
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,40 -1,24 +1,42 @@@
 +2.1.0
 + * (cqlsh) Fix COPY FROM handling of null/empty primary key
 +   values (CASSANDRA-7792)
 + * Fix ordering of static cells (CASSANDRA-7763)
- Merged from 2.0:
+ 2.0.10
+  * Fix ALTER clustering column type from DateType to TimestampType when
+    using DESC clustering order (CASSANRDA-7797)
 - * Stop inheriting liveRatio and liveRatioComputedAt from previous
 -   memtables (CASSANDRA-7796)
   * Throw EOFException if we run out of chunks in compressed datafile
     (CASSANDRA-7664)
 - * Throw InvalidRequestException when queries contain relations on entire
 -   collection columns (CASSANDRA-7506)
   * Fix PRSI handling of CQL3 row markers for row cleanup (CASSANDRA-7787)
 - * (cqlsh) enable CTRL-R history search with libedit (CASSANDRA-7577)
   * Fix dropping collection when it's the last regular column (CASSANDRA-7744)
   * Properly reject operations on list index with conditions (CASSANDRA-7499)
 - * (Hadoop) allow ACFRW to limit nodes to local DC (CASSANDRA-7252)
 + * Make StreamReceiveTask thread safe and gc friendly (CASSANDRA-7795)
 +Merged from 1.2:
 + * Validate empty cell names from counter updates (CASSANDRA-7798)
 +
 +
 +2.1.0-rc6
 + * Fix OOM issue from netty caching over time (CASSANDRA-7743)
 + * json2sstable couldn't import JSON for CQL table (CASSANDRA-7477)
 + * Invalidate all caches on table drop (CASSANDRA-7561)
 + * Skip strict endpoint selection for ranges if RF == nodes (CASSANRA-7765)
 + * Fix Thrift range filtering without 2ary index lookups (CASSANDRA-7741)
 + * Add tracing entries about concurrent range requests (CASSANDRA-7599)
 + * (cqlsh) Fix DESCRIBE for NTS keyspaces (CASSANDRA-7729)
 + * Remove netty buffer ref-counting (CASSANDRA-7735)
 + * Pass mutated cf to index updater for use by PRSI (CASSANDRA-7742)
 + * Include stress yaml example in release and deb (CASSANDRA-7717)
 + * workaround for netty issue causing corrupted data off the wire (CASSANDRA-7695)
 + * cqlsh DESC CLUSTER fails retrieving ring information (CASSANDRA-7687)
 + * Fix binding null values inside UDT (CASSANDRA-7685)
 + * Fix UDT field selection with empty fields (CASSANDRA-7670)
 + * Bogus deserialization of static cells from sstable (CASSANDRA-7684)
 + * Fix NPE on compaction leftover cleanup for dropped table (CASSANDRA-7770)
 +Merged from 2.0:
   * (cqlsh) Wait up to 10 sec for a tracing session (CASSANDRA-7222)
   * Fix NPE in FileCacheService.sizeInBytes (CASSANDRA-7756)
 - * (cqlsh) cqlsh should automatically disable tracing when selecting
 -   from system_traces (CASSANDRA-7641)
 - * (Hadoop) Add CqlOutputFormat (CASSANDRA-6927)
 - * Don't depend on cassandra config for nodetool ring (CASSANDRA-7508)
 - * (cqlsh) Fix failing cqlsh formatting tests (CASSANDRA-7703)
 + * Remove duplicates from StorageService.getJoiningNodes (CASSANDRA-7478)
 + * Clone token map outside of hot gossip loops (CASSANDRA-7758)
   * Fix MS expiring map timeout for Paxos messages (CASSANDRA-7752)
   * Do not flush on truncate if durable_writes is false (CASSANDRA-7750)
   * Give CRR a default input_cql Statement (CASSANDRA-7226)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d3450570/src/java/org/apache/cassandra/db/marshal/ReversedType.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d3450570/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/CQLTester.java
index 442c8b1,0000000..3dbff1e
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@@ -1,620 -1,0 +1,627 @@@
 +/*
 + * 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.db.ConsistencyLevel;
 +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.service.StorageService;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +
 +/**
 + * Base class for CQL tests.
 + */
 +public abstract class CQLTester
 +{
 +    protected static final Logger logger = LoggerFactory.getLogger(CQLTester.class);
 +
 +    private 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
 +        StorageService.optionalTasks.execute(new Runnable()
 +        {
 +            public void run()
 +            {
 +                try
 +                {
 +                    schemaChange(String.format("DROP TABLE %s.%s", KEYSPACE, tableToDrop));
 +
 +                    for (String typeName : typesToDrop)
 +                        schemaChange(String.format("DROP TYPE %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);
 +                    StorageService.tasks.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);
 +        }
 +    }
 +
 +    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 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 alterTable(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);
 +    }
 +
 +    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 UntypedResultSet execute(String query, Object... values) throws Throwable
 +    {
 +        if (currentTable == null)
 +            throw new RuntimeException("You must create a table first with createTable");
 +
 +        try
 +        {
 +            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), expected <%s> but got <%s>",
 +                                              i, j, column.name, formatValue(expectedByteValue, column.type), formatValue(actualValue, column.type)));
 +            }
 +        }
 +
 +        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
 +    {
 +        try
 +        {
 +            execute(query, values);
 +            Assert.fail("Query should be invalid but no error was thrown. Query is: " + query);
 +        }
 +        catch (SyntaxException | InvalidRequestException e)
 +        {
 +            // This is what we expect
 +        }
 +    }
 +
 +    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 base
 +        // 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;
 +            }
 +
 +            buffers[i] = typeFor(value).decompose(serializeTuples(value));
 +        }
 +        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)
 +        {
 +            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)
 +    {
 +        return bb == null ? "null" : 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 HashMap(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);
 +        }
 +
 +        if (value instanceof Set)
 +        {
 +            Set s = (Set)value;
 +            AbstractType elt = s.isEmpty() ? BytesType.instance : typeFor(s.iterator().next());
 +            return SetType.getInstance(elt);
 +        }
 +
 +        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);
 +        }
 +
 +        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();
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d3450570/test/unit/org/apache/cassandra/cql3/TypeTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/TypeTest.java
index f22d49b,b08ca2c..ec82d41
--- a/test/unit/org/apache/cassandra/cql3/TypeTest.java
+++ b/test/unit/org/apache/cassandra/cql3/TypeTest.java
@@@ -17,11 -17,88 +17,14 @@@
   */
  package org.apache.cassandra.cql3;
  
 -import org.apache.cassandra.SchemaLoader;
 -import org.apache.cassandra.db.ConsistencyLevel;
+ import org.apache.cassandra.exceptions.ConfigurationException;
 -import org.apache.cassandra.exceptions.RequestExecutionException;
 -import org.apache.cassandra.exceptions.RequestValidationException;
 -import org.apache.cassandra.gms.Gossiper;
 -import org.apache.cassandra.service.ClientState;
 -import org.apache.cassandra.service.QueryState;
 -import org.apache.cassandra.transport.messages.ResultMessage;
 -import org.apache.cassandra.utils.MD5Digest;
 -import org.junit.AfterClass;
 -import org.junit.BeforeClass;
  import org.junit.Test;
 -import org.slf4j.Logger;
 -import org.slf4j.LoggerFactory;
 -
 -import static org.apache.cassandra.cql3.QueryProcessor.process;
 -import static org.apache.cassandra.cql3.QueryProcessor.processInternal;
  import static org.junit.Assert.assertEquals;
++import static org.junit.Assert.assertTrue;
+ import static org.junit.Assert.fail;
  
 -public class TypeTest
 +public class TypeTest extends CQLTester
  {
 -    private static final Logger logger = LoggerFactory.getLogger(TypeTest.class);
 -    static ClientState clientState;
 -    static String keyspace = "cql3_type_test";
 -
 -    @BeforeClass
 -    public static void setUpClass() throws Throwable
 -    {
 -        SchemaLoader.loadSchema();
 -        executeSchemaChange("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}");
 -        clientState = ClientState.forInternalCalls();
 -    }
 -
 -    @AfterClass
 -    public static void stopGossiper()
 -    {
 -        Gossiper.instance.stop();
 -    }
 -
 -    private static void executeSchemaChange(String query) throws Throwable
 -    {
 -        try
 -        {
 -            process(String.format(query, keyspace), ConsistencyLevel.ONE);
 -        } catch (RuntimeException exc)
 -        {
 -            throw exc.getCause();
 -        }
 -    }
 -
 -    private static UntypedResultSet execute(String query) throws Throwable
 -    {
 -        try
 -        {
 -            return processInternal(String.format(query, keyspace));
 -        } catch (RuntimeException exc)
 -        {
 -            if (exc.getCause() != null)
 -                throw exc.getCause();
 -            throw exc;
 -        }
 -    }
 -
 -    private MD5Digest prepare(String query) throws RequestValidationException
 -    {
 -        ResultMessage.Prepared prepared = QueryProcessor.prepare(String.format(query, keyspace), clientState, false);
 -        return prepared.statementId;
 -    }
 -
 -    private UntypedResultSet executePrepared(MD5Digest statementId, QueryOptions options) throws RequestValidationException, RequestExecutionException
 -    {
 -        CQLStatement statement = QueryProcessor.instance.getPrepared(statementId);
 -        ResultMessage message = statement.executeInternal(QueryState.forInternalCalls(), options);
 -
 -        if (message instanceof ResultMessage.Rows)
 -            return new UntypedResultSet(((ResultMessage.Rows)message).result);
 -        else
 -            return null;
 -    }
 -
      @Test
      public void testNowToUUIDCompatibility() throws Throwable
      {
@@@ -43,9 -132,47 +46,41 @@@
      @Test
      public void testReversedTypeCompatibility() throws Throwable
      {
 -        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.uuid_now_reversed (a int, b timeuuid, PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
 -        String insert = "INSERT INTO %s.uuid_now_reversed (a, b) VALUES (0, now())";
 -        String select = "SELECT * FROM %s.uuid_now_reversed WHERE a=0 AND b < now()";
 -        execute(insert);
 -        UntypedResultSet results = execute(select);
 +        createTable("CREATE TABLE %s (a int, b timeuuid, PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
 +        execute("INSERT INTO %s (a, b) VALUES (0, now())");
 +        UntypedResultSet results = execute("SELECT * FROM %s WHERE a=0 AND b < now()");
          assertEquals(1, results.size());
 -
 -        executePrepared(prepare(insert), QueryOptions.DEFAULT);
 -        results = executePrepared(prepare(select), QueryOptions.DEFAULT);
 -        assertEquals(2, results.size());
      }
- }
+ 
+     @Test
+     // tests CASSANDRA-7797
+     public void testAlterReversedColumn() throws Throwable
+     {
 -        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.test_alter_reversed (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
 -        executeSchemaChange("ALTER TABLE %s.test_alter_reversed ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.TimestampType)'");
++        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
++        alterTable("ALTER TABLE %s ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.TimestampType)'");
+     }
+ 
+     @Test
+     public void testIncompatibleReversedTypes() throws Throwable
+     {
 -        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.test_incompatible_reversed (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
++        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
+         try
+         {
 -            executeSchemaChange("ALTER TABLE %s.test_incompatible_reversed ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.TimeUUIDType)'");
++            alterTable("ALTER TABLE %s ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.TimeUUIDType)'");
+             fail("Expected error for ALTER statement");
+         }
 -        catch (ConfigurationException e) { }
++        catch (RuntimeException e) { }
+     }
+ 
+     @Test
+     public void testReversedAndNonReversed() throws Throwable
+     {
 -        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.test_reversed_and_non_reversed (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b))");
++        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b))");
+         try
+         {
 -            executeSchemaChange("ALTER TABLE %s.test_reversed_and_non_reversed ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.DateType)'");
++            alterTable("ALTER TABLE %s ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.DateType)'");
+             fail("Expected error for ALTER statement");
+         }
 -        catch (ConfigurationException e) { }
++        catch (RuntimeException e) { }
+     }
 -}
++}