Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 3593C10508 for ; Wed, 19 Nov 2014 17:12:38 +0000 (UTC) Received: (qmail 16498 invoked by uid 500); 19 Nov 2014 17:12:37 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 16363 invoked by uid 500); 19 Nov 2014 17:12:37 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 16193 invoked by uid 99); 19 Nov 2014 17:12:37 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 19 Nov 2014 17:12:37 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 8A7869ABEB7; Wed, 19 Nov 2014 17:12:37 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: tylerhobbs@apache.org To: commits@cassandra.apache.org Date: Wed, 19 Nov 2014 17:12:38 -0000 Message-Id: <94bba81e0de544d6a2227158b53cd8ca@git.apache.org> In-Reply-To: <1a05b178171a4841957229ce662b421c@git.apache.org> References: <1a05b178171a4841957229ce662b421c@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [2/3] cassandra git commit: Merge branch 'cassandra-2.0' into cassandra-2.1 Merge branch 'cassandra-2.0' into cassandra-2.1 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/289314a6 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/289314a6 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/289314a6 Branch: refs/heads/trunk Commit: 289314a60f8a35e047a2073ef1e5ae003ada4250 Parents: 1b21aef 1945384 Author: Tyler Hobbs Authored: Wed Nov 19 11:09:23 2014 -0600 Committer: Tyler Hobbs Committed: Wed Nov 19 11:09:23 2014 -0600 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../apache/cassandra/cql3/ColumnIdentifier.java | 5 + .../cql3/statements/ModificationStatement.java | 2 +- .../cassandra/cql3/statements/RawSelector.java | 5 + .../cassandra/cql3/statements/Selectable.java | 20 ++ .../cassandra/cql3/statements/Selection.java | 78 ++++--- .../org/apache/cassandra/cql3/CQLTester.java | 7 + .../cassandra/cql3/SelectionOrderingTest.java | 233 +++++++++++++++++++ 8 files changed, 313 insertions(+), 38 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/289314a6/CHANGES.txt ---------------------------------------------------------------------- diff --cc CHANGES.txt index 2476d25,fff6d3a..80c6872 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,12 -1,5 +1,13 @@@ -2.0.12: +2.1.3 + * 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) +Merged from 2.0: + * Fix InvalidRequestException with ORDER BY (CASSANDRA-8286) * Disable SSLv3 for POODLE (CASSANDRA-8265) * Fix millisecond timestamps in Tracing (CASSANDRA-8297) * Include keyspace name in error message when there are insufficient http://git-wip-us.apache.org/repos/asf/cassandra/blob/289314a6/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/cql3/ColumnIdentifier.java index c1dcd87,2f3e481..1501479 --- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java +++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java @@@ -135,12 -103,17 +135,17 @@@ public class ColumnIdentifier implement ByteBuffer bufferName = ByteBufferUtil.bytes(text); for (ColumnDefinition def : cfm.partitionKeyColumns()) { - if (def.name.equals(bufferName)) + if (def.name.bytes.equals(bufferName)) return new ColumnIdentifier(text, true); } - return new ColumnIdentifier(cfm.comparator.fromString(rawText), text); + return new ColumnIdentifier(comparator.fromString(rawText), text); } + public boolean processesSelection() + { + return false; + } + @Override public final int hashCode() { http://git-wip-us.apache.org/repos/asf/cassandra/blob/289314a6/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java index 846ad3e,61f65c1..c32430a --- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java @@@ -610,12 -679,12 +610,12 @@@ public abstract class ModificationState // of batches for compatibility sakes). if (isBatch) { - names.addAll(cfDef.partitionKeys()); - names.addAll(cfDef.clusteringColumns()); + defs.addAll(cfm.partitionKeyColumns()); + defs.addAll(cfm.clusteringColumns()); } - for (ColumnIdentifier id : columnsWithConditions) - names.add(cfDef.get(id)); - selection = Selection.forColumns(new ArrayList<>(names)); + for (ColumnDefinition def : columnsWithConditions) + defs.add(def); - selection = Selection.forColumns(defs); ++ selection = Selection.forColumns(new ArrayList<>(defs)); } long now = System.currentTimeMillis(); http://git-wip-us.apache.org/repos/asf/cassandra/blob/289314a6/src/java/org/apache/cassandra/cql3/statements/Selectable.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/cql3/statements/Selectable.java index b7e8094,b7e3614..09fc661 --- a/src/java/org/apache/cassandra/cql3/statements/Selectable.java +++ b/src/java/org/apache/cassandra/cql3/statements/Selectable.java @@@ -109,41 -119,11 +119,51 @@@ public interface Selectabl preparedArgs.add(arg.prepare(cfm)); return new WithFunction(functionName, preparedArgs); } + + public boolean processesSelection() + { + return true; + } } } + + public static class WithFieldSelection implements Selectable + { + public final Selectable selected; + public final ColumnIdentifier field; + + public WithFieldSelection(Selectable selected, ColumnIdentifier field) + { + this.selected = selected; + this.field = field; + } + + @Override + public String toString() + { + return String.format("%s.%s", selected, field); + } + + public static class Raw implements Selectable.Raw + { + private final Selectable.Raw selected; + private final ColumnIdentifier.Raw field; + + public Raw(Selectable.Raw selected, ColumnIdentifier.Raw field) + { + this.selected = selected; + this.field = field; + } + + public WithFieldSelection prepare(CFMetaData cfm) + { + return new WithFieldSelection(selected.prepare(cfm), field.prepare(cfm)); + } ++ ++ public boolean processesSelection() ++ { ++ return true; ++ } + } + } } http://git-wip-us.apache.org/repos/asf/cassandra/blob/289314a6/src/java/org/apache/cassandra/cql3/statements/Selection.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/cql3/statements/Selection.java index 82efeef,407f7d9..5deda5f --- a/src/java/org/apache/cassandra/cql3/statements/Selection.java +++ b/src/java/org/apache/cassandra/cql3/statements/Selection.java @@@ -19,11 -19,8 +19,10 @@@ package org.apache.cassandra.cql3.state import java.nio.ByteBuffer; import java.util.ArrayList; - import java.util.Collection; import java.util.List; +import com.google.common.collect.Iterators; + import org.apache.cassandra.cql3.*; import org.apache.cassandra.cql3.functions.Function; import org.apache.cassandra.cql3.functions.Functions; @@@ -43,15 -36,15 +42,15 @@@ import org.apache.cassandra.utils.ByteB public abstract class Selection { - private final Collection columns; - private final List columns; - private final List metadata; ++ private final List columns; + private final ResultSet.Metadata metadata; private final boolean collectTimestamps; private final boolean collectTTLs; - protected Selection(Collection columns, List metadata, boolean collectTimestamps, boolean collectTTLs) - protected Selection(List columns, List metadata, boolean collectTimestamps, boolean collectTTLs) ++ protected Selection(List columns, List metadata, boolean collectTimestamps, boolean collectTTLs) { this.columns = columns; - this.metadata = metadata; + this.metadata = new ResultSet.Metadata(metadata); this.collectTimestamps = collectTimestamps; this.collectTTLs = collectTTLs; } @@@ -74,19 -68,12 +73,19 @@@ return new SimpleSelection(all, true); } - public static Selection forColumns(Collection columns) - public static Selection forColumns(List columns) ++ public static Selection forColumns(List columns) { return new SimpleSelection(columns, false); } - private static boolean selectionsNeedProcessing(List rawSelectors) + public int addColumnForOrdering(ColumnDefinition c) + { + columns.add(c); + metadata.addNonSerializedColumn(c); + return columns.size() - 1; + } + - private static boolean isUsingFunction(List rawSelectors) ++ private static boolean requiresProcessing(List rawSelectors) { for (RawSelector rawSelector : rawSelectors) { @@@ -209,18 -166,18 +208,16 @@@ returnType); } - private static ColumnSpecification makeAliasSpec(CFDefinition cfDef, AbstractType type, ColumnIdentifier alias) + private static ColumnSpecification makeAliasSpec(CFMetaData cfm, AbstractType type, ColumnIdentifier alias) { - return new ColumnSpecification(cfDef.cfm.ksName, cfDef.cfm.cfName, alias, type); + return new ColumnSpecification(cfm.ksName, cfm.cfName, alias, type); } - public static Selection fromSelectors(CFDefinition cfDef, List rawSelectors) throws InvalidRequestException + public static Selection fromSelectors(CFMetaData cfm, List rawSelectors) throws InvalidRequestException { - boolean usesFunction = isUsingFunction(rawSelectors); - - if (usesFunction) - boolean needsProcessing = selectionsNeedProcessing(rawSelectors); - - if (needsProcessing) ++ if (requiresProcessing(rawSelectors)) { - List names = new ArrayList(); + List defs = new ArrayList(); List metadata = new ArrayList(rawSelectors.size()); List selectors = new ArrayList(rawSelectors.size()); boolean collectTimestamps = false; @@@ -235,7 -192,7 +232,7 @@@ collectTTLs |= !((WritetimeOrTTLSelector)selector).isWritetime; } } - return new SelectionWithFunctions(defs, metadata, selectors, collectTimestamps, collectTTLs); - return new SelectionWithProcessing(names, metadata, selectors, collectTimestamps, collectTTLs); ++ return new SelectionWithProcessing(defs, metadata, selectors, collectTimestamps, collectTTLs); } else { @@@ -243,23 -200,38 +240,24 @@@ List metadata = new ArrayList(rawSelectors.size()); for (RawSelector rawSelector : rawSelectors) { - assert rawSelector.selectable instanceof ColumnIdentifier; - ColumnDefinition def = cfm.getColumnDefinition((ColumnIdentifier)rawSelector.selectable); + assert rawSelector.selectable instanceof ColumnIdentifier.Raw; - ColumnIdentifier id = ((ColumnIdentifier.Raw)rawSelector.selectable).prepare(cfDef.cfm); - CFDefinition.Name name = cfDef.get(id); - if (name == null) ++ ColumnIdentifier id = (ColumnIdentifier) rawSelector.selectable.prepare(cfm); ++ ColumnDefinition def = cfm.getColumnDefinition(id); + if (def == null) - throw new InvalidRequestException(String.format("Undefined name %s in selection clause", rawSelector.selectable)); + throw new InvalidRequestException(String.format("Undefined name %s in selection clause", id)); - names.add(name); - metadata.add(rawSelector.alias == null ? name : makeAliasSpec(cfDef, name.type, rawSelector.alias)); + defs.add(def); + metadata.add(rawSelector.alias == null ? def : makeAliasSpec(cfm, def.type, rawSelector.alias)); } - return new SimpleSelection(names, metadata, false); + return new SimpleSelection(defs, metadata, false); } } protected abstract List handleRow(ResultSetBuilder rs) throws InvalidRequestException; /** - * @return the list of CQL3 "regular" (the "COLUMN_METADATA" ones) column names to fetch. - */ - public List regularAndStaticColumnsToFetch() - { - List toFetch = new ArrayList(); - for (CFDefinition.Name name : columns) - { - if (name.kind == CFDefinition.Name.Kind.COLUMN_METADATA || name.kind == CFDefinition.Name.Kind.STATIC) - toFetch.add(name.name); - } - return toFetch; - } - - /** * @return the list of CQL3 columns value this SelectionClause needs. */ - public Collection getColumns() - public List getColumns() ++ public List getColumns() { return columns; } @@@ -350,12 -322,12 +348,12 @@@ { private final boolean isWildcard; - public SimpleSelection(Collection columns, boolean isWildcard) - public SimpleSelection(List columns, boolean isWildcard) ++ public SimpleSelection(List columns, boolean isWildcard) { this(columns, new ArrayList(columns), isWildcard); } - public SimpleSelection(Collection columns, List metadata, boolean isWildcard) - public SimpleSelection(List columns, List metadata, boolean isWildcard) ++ public SimpleSelection(List columns, List metadata, boolean isWildcard) { /* * In theory, even a simple selection could have multiple time the same column, so we @@@ -419,7 -406,7 +417,34 @@@ } } - private static class FunctionSelector implements Selector ++ private static class SelectionWithProcessing extends Selection ++ { ++ private final List selectors; ++ ++ public SelectionWithProcessing(List columns, List metadata, List selectors, boolean collectTimestamps, boolean collectTTLs) ++ { ++ super(columns, metadata, collectTimestamps, collectTTLs); ++ this.selectors = selectors; ++ } ++ ++ protected List handleRow(ResultSetBuilder rs) throws InvalidRequestException ++ { ++ List result = new ArrayList<>(); ++ for (Selector selector : selectors) ++ result.add(selector.compute(rs)); ++ return result; ++ } ++ ++ @Override ++ public int addColumnForOrdering(ColumnDefinition c) ++ { ++ int index = super.addColumnForOrdering(c); ++ selectors.add(new SimpleSelector(c.name.toString(), index, c.type)); ++ return index; ++ } ++ } ++ + private static class FunctionSelector extends Selector { private final Function fun; private final List argSelectors; http://git-wip-us.apache.org/repos/asf/cassandra/blob/289314a6/test/unit/org/apache/cassandra/cql3/CQLTester.java ---------------------------------------------------------------------- diff --cc test/unit/org/apache/cassandra/cql3/CQLTester.java index aacfb4b,0000000..a110af6 mode 100644,000000..100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@@ -1,742 -1,0 +1,749 @@@ +/* + * 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.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; +import org.apache.cassandra.service.StorageService; + +/** + * 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 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 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 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); + 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); + } + } + + 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(); + } + } + + 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 meta = result.metadata(); + Iterator 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)); + } + } + } + + protected void assertInvalidSyntax(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 + } + } + + 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/289314a6/test/unit/org/apache/cassandra/cql3/SelectionOrderingTest.java ---------------------------------------------------------------------- diff --cc test/unit/org/apache/cassandra/cql3/SelectionOrderingTest.java index 0000000,305d91e..301aaf4 mode 000000,100644..100644 --- a/test/unit/org/apache/cassandra/cql3/SelectionOrderingTest.java +++ b/test/unit/org/apache/cassandra/cql3/SelectionOrderingTest.java @@@ -1,0 -1,452 +1,233 @@@ + /* + * 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.apache.cassandra.SchemaLoader; -import org.apache.cassandra.db.ConsistencyLevel; -import org.apache.cassandra.gms.Gossiper; -import org.apache.cassandra.service.ClientState; -import org.junit.AfterClass; -import org.junit.BeforeClass; + import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + -import java.util.Iterator; - -import static org.apache.cassandra.cql3.QueryProcessor.process; -import static org.apache.cassandra.cql3.QueryProcessor.processInternal; -import static org.junit.Assert.assertEquals; - -public class SelectionOrderingTest ++public class SelectionOrderingTest extends CQLTester + { - private static final Logger logger = LoggerFactory.getLogger(SelectWithTokenFunctionTest.class); - static ClientState clientState; - static String keyspace = "select_with_ordering_test"; - - @BeforeClass - public static void setUpClass() throws Throwable - { - SchemaLoader.loadSchema(); - executeSchemaChange("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}"); - executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.single_clustering (a int, b int, c int, PRIMARY KEY (a, b))"); - executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.single_clustering_desc (a int, b int, c int, PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)"); - executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.multiple_clustering (a int, b int, c int, d int, PRIMARY KEY (a, b, c))"); - 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 ++ @Test ++ public void testNormalSelectionOrderSingleClustering() throws Throwable + { - try ++ for (String descOption : new String[]{"", " WITH CLUSTERING ORDER BY (b DESC)"}) + { - return processInternal(String.format(query, keyspace)); - } - catch (RuntimeException exc) - { - if (exc.getCause() != null) - throw exc.getCause(); - throw exc; ++ createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + descOption); ++ execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0); ++ execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1); ++ execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 2); ++ ++ assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b ASC", 0), ++ row(0, 0, 0), ++ row(0, 1, 1), ++ row(0, 2, 2) ++ ); ++ ++ assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC", 0), ++ row(0, 2, 2), ++ row(0, 1, 1), ++ row(0, 0, 0) ++ ); ++ ++ // order by the only column in the selection ++ assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b ASC", 0), ++ row(0), row(1), row(2)); ++ ++ assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b DESC", 0), ++ row(2), row(1), row(0)); ++ ++ // order by a column not in the selection ++ assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC", 0), ++ row(0), row(1), row(2)); ++ ++ assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC", 0), ++ row(2), row(1), row(0)); + } + } + + @Test - public void testNormalSelectionOrderSingleClustering() throws Throwable ++ public void testFunctionSelectionOrderSingleClustering() throws Throwable + { - for (String descOption : new String[]{"", "_desc"}) ++ for (String descOption : new String[]{"", " WITH CLUSTERING ORDER BY (b DESC)"}) + { - execute("INSERT INTO %s.single_clustering" + descOption + " (a, b, c) VALUES (0, 0, 0)"); - execute("INSERT INTO %s.single_clustering" + descOption + " (a, b, c) VALUES (0, 1, 1)"); - execute("INSERT INTO %s.single_clustering" + descOption + " (a, b, c) VALUES (0, 2, 2)"); - - try - { - UntypedResultSet results = execute("SELECT * FROM %s.single_clustering" + descOption + " WHERE a=0 ORDER BY b ASC"); - assertEquals(3, results.size()); - Iterator rows = results.iterator(); - for (int i = 0; i < 3; i++) - assertEquals(i, rows.next().getInt("b")); - - results = execute("SELECT * FROM %s.single_clustering" + descOption + " WHERE a=0 ORDER BY b DESC"); - assertEquals(3, results.size()); - rows = results.iterator(); - for (int i = 2; i >= 0; i--) - assertEquals(i, rows.next().getInt("b")); - - // order by the only column in the selection - results = execute("SELECT b FROM %s.single_clustering" + descOption + " WHERE a=0 ORDER BY b ASC"); - assertEquals(3, results.size()); - rows = results.iterator(); - for (int i = 0; i < 3; i++) - assertEquals(i, rows.next().getInt("b")); - - results = execute("SELECT b FROM %s.single_clustering" + descOption + " WHERE a=0 ORDER BY b DESC"); - assertEquals(3, results.size()); - rows = results.iterator(); - for (int i = 2; i >= 0; i--) - assertEquals(i, rows.next().getInt("b")); - - // order by a column not in the selection - results = execute("SELECT c FROM %s.single_clustering" + descOption + " WHERE a=0 ORDER BY b ASC"); - assertEquals(3, results.size()); - rows = results.iterator(); - for (int i = 0; i < 3; i++) - assertEquals(i, rows.next().getInt("c")); - - results = execute("SELECT c FROM %s.single_clustering" + descOption + " WHERE a=0 ORDER BY b DESC"); - assertEquals(3, results.size()); - rows = results.iterator(); - for (int i = 2; i >= 0; i--) - assertEquals(i, rows.next().getInt("c")); - } - finally - { - execute("DELETE FROM %s.single_clustering" + descOption + " WHERE a = 0"); - } ++ createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + descOption); ++ execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0); ++ execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1); ++ execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 2); ++ ++ // order by the only column in the selection ++ assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0), ++ row(0), row(1), row(2)); ++ ++ assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0), ++ row(2), row(1), row(0)); ++ ++ // order by a column not in the selection ++ assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC", 0), ++ row(0), row(1), row(2)); ++ ++ assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC", 0), ++ row(2), row(1), row(0)); ++ ++ assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c ASC", 0); ++ assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c DESC", 0); + } + } + + @Test - public void testFunctionSelectionOrderSingleClustering() throws Throwable ++ public void testFieldSelectionOrderSingleClustering() throws Throwable + { - for (String descOption : new String[]{"", "_desc"}) ++ String type = createType("CREATE TYPE %s (a int)"); ++ ++ for (String descOption : new String[]{"", " WITH CLUSTERING ORDER BY (b DESC)"}) + { - execute("INSERT INTO %s.single_clustering" + descOption + " (a, b, c) VALUES (0, 0, 0)"); - execute("INSERT INTO %s.single_clustering" + descOption + " (a, b, c) VALUES (0, 1, 1)"); - execute("INSERT INTO %s.single_clustering" + descOption + " (a, b, c) VALUES (0, 2, 2)"); - - try - { - // order by a column in the selection (wrapped in a function) - UntypedResultSet results = execute("SELECT blobAsInt(intAsBlob(b)) as col FROM %s.single_clustering" + descOption + " WHERE a=0 ORDER BY b ASC"); - assertEquals(3, results.size()); - Iterator rows = results.iterator(); - for (int i = 0; i < 3; i++) - assertEquals(i, rows.next().getInt("col")); - - results = execute("SELECT blobAsInt(intAsBlob(b)) as col FROM %s.single_clustering" + descOption + " WHERE a=0 ORDER BY b DESC"); - assertEquals(3, results.size()); - rows = results.iterator(); - for (int i = 2; i >= 0; i--) - assertEquals(i, rows.next().getInt("col")); - - // order by a column in the selection, plus the column wrapped in a function - results = execute("SELECT b, blobAsInt(intAsBlob(b)) as col FROM %s.single_clustering" + descOption + " WHERE a=0 ORDER BY b ASC"); - assertEquals(3, results.size()); - rows = results.iterator(); - for (int i = 0; i < 3; i++) - { - UntypedResultSet.Row row = rows.next(); - assertEquals(i, row.getInt("b")); - assertEquals(i, row.getInt("col")); - } - - results = execute("SELECT b, blobAsInt(intAsBlob(b)) as col FROM %s.single_clustering" + descOption + " WHERE a=0 ORDER BY b DESC"); - assertEquals(3, results.size()); - rows = results.iterator(); - for (int i = 2; i >= 0; i--) - { - UntypedResultSet.Row row = rows.next(); - assertEquals(i, row.getInt("b")); - assertEquals(i, row.getInt("col")); - } - - // order by a column not in the selection (wrapped in a function) - results = execute("SELECT blobAsInt(intAsBlob(c)) as col FROM %s.single_clustering" + descOption + " WHERE a=0 ORDER BY b ASC"); - assertEquals(3, results.size()); - rows = results.iterator(); - for (int i = 0; i < 3; i++) - assertEquals(i, rows.next().getInt("col")); - - results = execute("SELECT blobAsInt(intAsBlob(c)) as col FROM %s.single_clustering" + descOption + " WHERE a=0 ORDER BY b DESC"); - assertEquals(3, results.size()); - rows = results.iterator(); - for (int i = 2; i >= 0; i--) - assertEquals(i, rows.next().getInt("col")); - } - finally - { - execute("DELETE FROM %s.single_clustering" + descOption + " WHERE a = 0"); - } ++ createTable("CREATE TABLE %s (a int, b int, c frozen<" + type + " >, PRIMARY KEY (a, b))" + descOption); ++ execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 0, 0); ++ execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 1, 1); ++ execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 2, 2); ++ ++ // order by a column not in the selection ++ assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b ASC", 0), ++ row(0), row(1), row(2)); ++ ++ assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b DESC", 0), ++ row(2), row(1), row(0)); ++ ++ assertRows(execute("SELECT blobAsInt(intAsBlob(c.a)) FROM %s WHERE a=? ORDER BY b DESC", 0), ++ row(2), row(1), row(0)); ++ dropTable("DROP TABLE %s"); + } + } + + @Test + public void testNormalSelectionOrderMultipleClustering() throws Throwable + { - execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)"); - execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)"); - execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 2, 2)"); - execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 0, 3)"); - execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 4)"); - execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 2, 5)"); - try - { - UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 ORDER BY b ASC"); - assertEquals(6, results.size()); - Iterator rows = results.iterator(); - assertEquals(0, rows.next().getInt("d")); - assertEquals(1, rows.next().getInt("d")); - assertEquals(2, rows.next().getInt("d")); - assertEquals(3, rows.next().getInt("d")); - assertEquals(4, rows.next().getInt("d")); - assertEquals(5, rows.next().getInt("d")); - - results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 ORDER BY b DESC"); - assertEquals(6, results.size()); - rows = results.iterator(); - assertEquals(5, rows.next().getInt("d")); - assertEquals(4, rows.next().getInt("d")); - assertEquals(3, rows.next().getInt("d")); - assertEquals(2, rows.next().getInt("d")); - assertEquals(1, rows.next().getInt("d")); - assertEquals(0, rows.next().getInt("d")); - - results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 ORDER BY b DESC, c DESC"); - assertEquals(6, results.size()); - rows = results.iterator(); - assertEquals(5, rows.next().getInt("d")); - assertEquals(4, rows.next().getInt("d")); - assertEquals(3, rows.next().getInt("d")); - assertEquals(2, rows.next().getInt("d")); - assertEquals(1, rows.next().getInt("d")); - assertEquals(0, rows.next().getInt("d")); - - // select and order by b - results = execute("SELECT b FROM %s.multiple_clustering WHERE a=0 ORDER BY b ASC"); - assertEquals(6, results.size()); - rows = results.iterator(); - assertEquals(0, rows.next().getInt("b")); - assertEquals(0, rows.next().getInt("b")); - assertEquals(0, rows.next().getInt("b")); - assertEquals(1, rows.next().getInt("b")); - assertEquals(1, rows.next().getInt("b")); - assertEquals(1, rows.next().getInt("b")); - - results = execute("SELECT b FROM %s.multiple_clustering WHERE a=0 ORDER BY b DESC"); - assertEquals(6, results.size()); - rows = results.iterator(); - assertEquals(1, rows.next().getInt("b")); - assertEquals(1, rows.next().getInt("b")); - assertEquals(1, rows.next().getInt("b")); - assertEquals(0, rows.next().getInt("b")); - assertEquals(0, rows.next().getInt("b")); - assertEquals(0, rows.next().getInt("b")); - - // select c, order by b - results = execute("SELECT c FROM %s.multiple_clustering WHERE a=0 ORDER BY b ASC"); - rows = results.iterator(); - assertEquals(0, rows.next().getInt("c")); - assertEquals(1, rows.next().getInt("c")); - assertEquals(2, rows.next().getInt("c")); - assertEquals(0, rows.next().getInt("c")); - assertEquals(1, rows.next().getInt("c")); - assertEquals(2, rows.next().getInt("c")); - - results = execute("SELECT c FROM %s.multiple_clustering WHERE a=0 ORDER BY b DESC"); - assertEquals(6, results.size()); - rows = results.iterator(); - assertEquals(2, rows.next().getInt("c")); - assertEquals(1, rows.next().getInt("c")); - assertEquals(0, rows.next().getInt("c")); - assertEquals(2, rows.next().getInt("c")); - assertEquals(1, rows.next().getInt("c")); - assertEquals(0, rows.next().getInt("c")); - - // select c, order by b, c - results = execute("SELECT c FROM %s.multiple_clustering WHERE a=0 ORDER BY b ASC, c ASC"); - rows = results.iterator(); - assertEquals(0, rows.next().getInt("c")); - assertEquals(1, rows.next().getInt("c")); - assertEquals(2, rows.next().getInt("c")); - assertEquals(0, rows.next().getInt("c")); - assertEquals(1, rows.next().getInt("c")); - assertEquals(2, rows.next().getInt("c")); - - results = execute("SELECT c FROM %s.multiple_clustering WHERE a=0 ORDER BY b DESC, c DESC"); - assertEquals(6, results.size()); - rows = results.iterator(); - assertEquals(2, rows.next().getInt("c")); - assertEquals(1, rows.next().getInt("c")); - assertEquals(0, rows.next().getInt("c")); - assertEquals(2, rows.next().getInt("c")); - assertEquals(1, rows.next().getInt("c")); - assertEquals(0, rows.next().getInt("c")); - - // select d, order by b, c - results = execute("SELECT d FROM %s.multiple_clustering WHERE a=0 ORDER BY b ASC, c ASC"); - rows = results.iterator(); - assertEquals(0, rows.next().getInt("d")); - assertEquals(1, rows.next().getInt("d")); - assertEquals(2, rows.next().getInt("d")); - assertEquals(3, rows.next().getInt("d")); - assertEquals(4, rows.next().getInt("d")); - assertEquals(5, rows.next().getInt("d")); - - results = execute("SELECT d FROM %s.multiple_clustering WHERE a=0 ORDER BY b DESC, c DESC"); - assertEquals(6, results.size()); - rows = results.iterator(); - assertEquals(5, rows.next().getInt("d")); - assertEquals(4, rows.next().getInt("d")); - assertEquals(3, rows.next().getInt("d")); - assertEquals(2, rows.next().getInt("d")); - assertEquals(1, rows.next().getInt("d")); - assertEquals(0, rows.next().getInt("d")); - } - finally - { - execute("DELETE FROM %s.multiple_clustering WHERE a = 0"); - } ++ createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))"); ++ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0); ++ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1); ++ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 2, 2); ++ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 3); ++ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 4); ++ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 2, 5); ++ ++ assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b ASC", 0), ++ row(0, 0, 0, 0), ++ row(0, 0, 1, 1), ++ row(0, 0, 2, 2), ++ row(0, 1, 0, 3), ++ row(0, 1, 1, 4), ++ row(0, 1, 2, 5) ++ ); ++ ++ assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC", 0), ++ row(0, 1, 2, 5), ++ row(0, 1, 1, 4), ++ row(0, 1, 0, 3), ++ row(0, 0, 2, 2), ++ row(0, 0, 1, 1), ++ row(0, 0, 0, 0) ++ ); ++ ++ assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0), ++ row(0, 1, 2, 5), ++ row(0, 1, 1, 4), ++ row(0, 1, 0, 3), ++ row(0, 0, 2, 2), ++ row(0, 0, 1, 1), ++ row(0, 0, 0, 0) ++ ); ++ ++ assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c ASC", 0); ++ assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c DESC", 0); ++ assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY b ASC, c DESC", 0); ++ assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY b DESC, c ASC", 0); ++ assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY d ASC", 0); ++ ++ // select and order by b ++ assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b ASC", 0), ++ row(0), row(0), row(0), row(1), row(1), row(1)); ++ assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b DESC", 0), ++ row(1), row(1), row(1), row(0), row(0), row(0)); ++ ++ // select c, order by b ++ assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC", 0), ++ row(0), row(1), row(2), row(0), row(1), row(2)); ++ assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC", 0), ++ row(2), row(1), row(0), row(2), row(1), row(0)); ++ ++ // select c, order by b, c ++ assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0), ++ row(0), row(1), row(2), row(0), row(1), row(2)); ++ assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0), ++ row(2), row(1), row(0), row(2), row(1), row(0)); ++ ++ // select d, order by b, c ++ assertRows(execute("SELECT d FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0), ++ row(0), row(1), row(2), row(3), row(4), row(5)); ++ assertRows(execute("SELECT d FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0), ++ row(5), row(4), row(3), row(2), row(1), row(0)); + } + + @Test + public void testFunctionSelectionOrderMultipleClustering() throws Throwable + { - execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)"); - execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)"); - execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 2, 2)"); - execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 0, 3)"); - execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 4)"); - execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 2, 5)"); - try - { - // select function of b, order by b - UntypedResultSet results = execute("SELECT blobAsInt(intAsBlob(b)) as col FROM %s.multiple_clustering WHERE a=0 ORDER BY b ASC"); - assertEquals(6, results.size()); - Iterator rows = results.iterator(); - assertEquals(0, rows.next().getInt("col")); - assertEquals(0, rows.next().getInt("col")); - assertEquals(0, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - - results = execute("SELECT blobAsInt(intAsBlob(b)) as col FROM %s.multiple_clustering WHERE a=0 ORDER BY b DESC"); - assertEquals(6, results.size()); - rows = results.iterator(); - assertEquals(1, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - assertEquals(0, rows.next().getInt("col")); - assertEquals(0, rows.next().getInt("col")); - assertEquals(0, rows.next().getInt("col")); - - // select b and function of b, order by b - results = execute("SELECT b, blobAsInt(intAsBlob(b)) as col FROM %s.multiple_clustering WHERE a=0 ORDER BY b ASC"); - assertEquals(6, results.size()); - rows = results.iterator(); - assertEquals(0, rows.next().getInt("col")); - assertEquals(0, rows.next().getInt("col")); - assertEquals(0, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - - results = execute("SELECT b, blobAsInt(intAsBlob(b)) as col FROM %s.multiple_clustering WHERE a=0 ORDER BY b DESC"); - assertEquals(6, results.size()); - rows = results.iterator(); - assertEquals(1, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - assertEquals(0, rows.next().getInt("col")); - assertEquals(0, rows.next().getInt("col")); - assertEquals(0, rows.next().getInt("col")); - - // select c, order by b - results = execute("SELECT blobAsInt(intAsBlob(c)) as col FROM %s.multiple_clustering WHERE a=0 ORDER BY b ASC"); - rows = results.iterator(); - assertEquals(0, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - assertEquals(2, rows.next().getInt("col")); - assertEquals(0, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - assertEquals(2, rows.next().getInt("col")); - - results = execute("SELECT blobAsInt(intAsBlob(c)) as col FROM %s.multiple_clustering WHERE a=0 ORDER BY b DESC"); - assertEquals(6, results.size()); - rows = results.iterator(); - assertEquals(2, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - assertEquals(0, rows.next().getInt("col")); - assertEquals(2, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - assertEquals(0, rows.next().getInt("col")); - - // select c, order by b, c - results = execute("SELECT blobAsInt(intAsBlob(c)) as col FROM %s.multiple_clustering WHERE a=0 ORDER BY b ASC, c ASC"); - rows = results.iterator(); - assertEquals(0, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - assertEquals(2, rows.next().getInt("col")); - assertEquals(0, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - assertEquals(2, rows.next().getInt("col")); - - results = execute("SELECT blobAsInt(intAsBlob(c)) as col FROM %s.multiple_clustering WHERE a=0 ORDER BY b DESC, c DESC"); - assertEquals(6, results.size()); - rows = results.iterator(); - assertEquals(2, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - assertEquals(0, rows.next().getInt("col")); - assertEquals(2, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - assertEquals(0, rows.next().getInt("col")); - - // select d, order by b, c - results = execute("SELECT blobAsInt(intAsBlob(d)) as col FROM %s.multiple_clustering WHERE a=0 ORDER BY b ASC, c ASC"); - rows = results.iterator(); - assertEquals(0, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - assertEquals(2, rows.next().getInt("col")); - assertEquals(3, rows.next().getInt("col")); - assertEquals(4, rows.next().getInt("col")); - assertEquals(5, rows.next().getInt("col")); - - results = execute("SELECT blobAsInt(intAsBlob(d)) as col FROM %s.multiple_clustering WHERE a=0 ORDER BY b DESC, c DESC"); - assertEquals(6, results.size()); - rows = results.iterator(); - assertEquals(5, rows.next().getInt("col")); - assertEquals(4, rows.next().getInt("col")); - assertEquals(3, rows.next().getInt("col")); - assertEquals(2, rows.next().getInt("col")); - assertEquals(1, rows.next().getInt("col")); - assertEquals(0, rows.next().getInt("col")); - } - finally - { - execute("DELETE FROM %s.multiple_clustering WHERE a = 0"); - } ++ createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))"); ++ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0); ++ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1); ++ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 2, 2); ++ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 3); ++ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 4); ++ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 2, 5); ++ ++ assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY c ASC", 0); ++ assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY c DESC", 0); ++ assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC, c DESC", 0); ++ assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC, c ASC", 0); ++ assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY d ASC", 0); ++ ++ // select and order by b ++ assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0), ++ row(0), row(0), row(0), row(1), row(1), row(1)); ++ assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0), ++ row(1), row(1), row(1), row(0), row(0), row(0)); ++ ++ assertRows(execute("SELECT b, blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0), ++ row(0, 0), row(0, 0), row(0, 0), row(1, 1), row(1, 1), row(1, 1)); ++ assertRows(execute("SELECT b, blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0), ++ row(1, 1), row(1, 1), row(1, 1), row(0, 0), row(0, 0), row(0, 0)); ++ ++ // select c, order by b ++ assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC", 0), ++ row(0), row(1), row(2), row(0), row(1), row(2)); ++ assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC", 0), ++ row(2), row(1), row(0), row(2), row(1), row(0)); ++ ++ // select c, order by b, c ++ assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0), ++ row(0), row(1), row(2), row(0), row(1), row(2)); ++ assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0), ++ row(2), row(1), row(0), row(2), row(1), row(0)); ++ ++ // select d, order by b, c ++ assertRows(execute("SELECT blobAsInt(intAsBlob(d)) FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0), ++ row(0), row(1), row(2), row(3), row(4), row(5)); ++ assertRows(execute("SELECT blobAsInt(intAsBlob(d)) FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0), ++ row(5), row(4), row(3), row(2), row(1), row(0)); ++ + } + }