http://git-wip-us.apache.org/repos/asf/cassandra/blob/3bdcaa33/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java index b7db989..7311069 100644 --- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java +++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java @@ -24,14 +24,25 @@ import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.cassandra.concurrent.NamedThreadFactory; import org.apache.cassandra.config.Schema; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.db.compaction.OperationType; +import org.apache.cassandra.db.Mutation; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.rows.RowIterators; +import org.apache.cassandra.db.rows.Unfiltered; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.db.rows.UnfilteredRowIterators; +import org.apache.cassandra.io.sstable.ISSTableScanner; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.format.SSTableWriter; +import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.Pair; import org.apache.cassandra.utils.concurrent.Refs; @@ -41,6 +52,8 @@ import org.apache.cassandra.utils.concurrent.Refs; */ public class StreamReceiveTask extends StreamTask { + private static final Logger logger = LoggerFactory.getLogger(StreamReceiveTask.class); + private static final ExecutorService executor = Executors.newCachedThreadPool(new NamedThreadFactory("StreamReceiveTask")); // number of files to receive @@ -120,21 +133,69 @@ public class StreamReceiveTask extends StreamTask return; } ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right); + boolean hasMaterializedViews = cfs.materializedViewManager.allViews().iterator().hasNext(); + + try + { + List<SSTableReader> readers = new ArrayList<>(); + for (SSTableWriter writer : task.sstables) + { + SSTableReader reader = writer.finish(true); + readers.add(reader); + task.txn.update(reader, false); + } - List<SSTableReader> readers = new ArrayList<>(); - for (SSTableWriter writer : task.sstables) - readers.add(writer.finish(true)); - task.txn.finish(); - task.sstables.clear(); + task.sstables.clear(); - try (Refs<SSTableReader> refs = Refs.ref(readers)) + try (Refs<SSTableReader> refs = Refs.ref(readers)) + { + //We have a special path for Materialized view. + //Since the MV requires cleaning up any pre-existing state, we must put + //all partitions through the same write path as normal mutations. + //This also ensures any 2is are also updated + if (hasMaterializedViews) + { + for (SSTableReader reader : readers) + { + try (ISSTableScanner scanner = reader.getScanner()) + { + while (scanner.hasNext()) + { + try (UnfilteredRowIterator rowIterator = scanner.next()) + { + new Mutation(PartitionUpdate.fromIterator(rowIterator)).apply(); + } + } + } + } + } + else + { + task.txn.finish(); + + // add sstables and build secondary indexes + cfs.addSSTables(readers); + cfs.indexManager.maybeBuildSecondaryIndexes(readers, cfs.indexManager.allIndexesNames()); + } + } + catch (Throwable t) + { + logger.error("Error applying streamed sstable: ", t); + + JVMStabilityInspector.inspectThrowable(t); + } + finally + { + //We don't keep the streamed sstables since we've applied them manually + //So we abort the txn and delete the streamed sstables + if (hasMaterializedViews) + task.txn.abort(); + } + } + finally { - // add sstables and build secondary indexes - cfs.addSSTables(readers); - cfs.indexManager.maybeBuildSecondaryIndexes(readers, cfs.indexManager.allIndexesNames()); + task.session.taskCompleted(task); } - - task.session.taskCompleted(task); } }
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3bdcaa33/src/java/org/apache/cassandra/thrift/CassandraServer.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java index b38f58e..0674c62 100644 --- a/src/java/org/apache/cassandra/thrift/CassandraServer.java +++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java @@ -805,6 +805,9 @@ public class CassandraServer implements Cassandra.Iface cState.hasColumnFamilyAccess(keyspace, column_parent.column_family, Permission.MODIFY); CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, column_parent.column_family, false); + if (metadata.isMaterializedView()) + throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot modify Materialized Views directly"); + ThriftValidation.validateKey(metadata, key); ThriftValidation.validateColumnParent(metadata, column_parent); // SuperColumn field is usually optional, but not when we're inserting @@ -898,6 +901,9 @@ public class CassandraServer implements Cassandra.Iface cState.hasColumnFamilyAccess(keyspace, column_family, Permission.SELECT); CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, column_family, false); + if (metadata.isMaterializedView()) + throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot modify Materialized Views directly"); + ThriftValidation.validateKey(metadata, key); if (metadata.isSuper()) throw new org.apache.cassandra.exceptions.InvalidRequestException("CAS does not support supercolumns"); @@ -1090,6 +1096,9 @@ public class CassandraServer implements Cassandra.Iface cState.hasColumnFamilyAccess(keyspace, cfName, Permission.MODIFY); CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, cfName); + if (metadata.isMaterializedView()) + throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot modify Materialized Views directly"); + ThriftValidation.validateKey(metadata, key); if (metadata.isCounter()) ThriftConversion.fromThrift(consistency_level).validateCounterForWrite(metadata); @@ -1303,6 +1312,9 @@ public class CassandraServer implements Cassandra.Iface cState.hasColumnFamilyAccess(keyspace, column_path.column_family, Permission.MODIFY); CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, column_path.column_family, isCommutativeOp); + if (metadata.isMaterializedView()) + throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot modify Materialized Views directly"); + ThriftValidation.validateKey(metadata, key); ThriftValidation.validateColumnPathOrParent(metadata, column_path); if (isCommutativeOp) @@ -1872,6 +1884,11 @@ public class CassandraServer implements Cassandra.Iface { String keyspace = cState.getKeyspace(); cState.hasColumnFamilyAccess(keyspace, column_family, Permission.DROP); + + CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, column_family); + if (metadata.isMaterializedView()) + throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot drop Materialized Views from Thrift"); + MigrationManager.announceColumnFamilyDrop(keyspace, column_family); return Schema.instance.getVersion().toString(); } @@ -1982,6 +1999,11 @@ public class CassandraServer implements Cassandra.Iface if (oldCfm == null) throw new InvalidRequestException("Could not find table definition to modify."); + if (oldCfm.isMaterializedView()) + throw new InvalidRequestException("Cannot modify Materialized View table " + oldCfm.cfName + " as it may break the schema. You should use cqlsh to modify Materialized View tables instead."); + if (!oldCfm.getMaterializedViews().isEmpty()) + throw new InvalidRequestException("Cannot modify table with Materialized View " + oldCfm.cfName + " as it may break the schema. You should use cqlsh to modify tables with Materialized Views instead."); + if (!oldCfm.isThriftCompatible()) throw new InvalidRequestException("Cannot modify CQL3 table " + oldCfm.cfName + " as it may break the schema. You should use cqlsh to modify CQL3 tables instead."); @@ -2009,6 +2031,9 @@ public class CassandraServer implements Cassandra.Iface { String keyspace = cState.getKeyspace(); cState.hasColumnFamilyAccess(keyspace, cfname, Permission.MODIFY); + CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, cfname, true); + if (metadata.isMaterializedView()) + throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot truncate Materialized Views"); if (startSessionIfRequested()) { @@ -2094,6 +2119,9 @@ public class CassandraServer implements Cassandra.Iface cState.hasColumnFamilyAccess(keyspace, column_parent.column_family, Permission.MODIFY); CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, column_parent.column_family, true); + if (metadata.isMaterializedView()) + throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot modify Materialized Views directly"); + ThriftValidation.validateKey(metadata, key); ThriftConversion.fromThrift(consistency_level).validateCounterForWrite(metadata); ThriftValidation.validateColumnParent(metadata, column_parent); http://git-wip-us.apache.org/repos/asf/cassandra/blob/3bdcaa33/src/java/org/apache/cassandra/thrift/ThriftConversion.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/thrift/ThriftConversion.java b/src/java/org/apache/cassandra/thrift/ThriftConversion.java index c9e5062..20d6eba 100644 --- a/src/java/org/apache/cassandra/thrift/ThriftConversion.java +++ b/src/java/org/apache/cassandra/thrift/ThriftConversion.java @@ -274,7 +274,10 @@ public class ThriftConversion subComparator, defaultValidator); - CFMetaData newCFMD = CFMetaData.create(cf_def.keyspace, cf_def.name, cfId, isDense, isCompound, isSuper, isCounter, defs); + // We do not allow Thrift materialized views, so we always set it to false + boolean isMaterializedView = false; + + CFMetaData newCFMD = CFMetaData.create(cf_def.keyspace, cf_def.name, cfId, isDense, isCompound, isSuper, isCounter, isMaterializedView, defs); if (cf_def.isSetGc_grace_seconds()) newCFMD.gcGraceSeconds(cf_def.gc_grace_seconds); http://git-wip-us.apache.org/repos/asf/cassandra/blob/3bdcaa33/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java index d9c268b..5354b43 100644 --- a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java +++ b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java @@ -118,6 +118,7 @@ public class NativeSSTableLoaderClient extends SSTableLoader.Client boolean isCounter = flags.contains(CFMetaData.Flag.COUNTER); boolean isDense = flags.contains(CFMetaData.Flag.DENSE); boolean isCompound = flags.contains(CFMetaData.Flag.COMPOUND); + boolean isMaterializedView = flags.contains(CFMetaData.Flag.MATERIALIZEDVIEW); String columnsQuery = String.format("SELECT * FROM %s.%s WHERE keyspace_name = ? AND table_name = ?", SchemaKeyspace.NAME, @@ -127,7 +128,7 @@ public class NativeSSTableLoaderClient extends SSTableLoader.Client for (Row colRow : session.execute(columnsQuery, keyspace, name)) defs.add(createDefinitionFromRow(colRow, keyspace, name)); - tables.put(name, CFMetaData.create(keyspace, name, id, isDense, isCompound, isSuper, isCounter, defs)); + tables.put(name, CFMetaData.create(keyspace, name, id, isDense, isCompound, isSuper, isCounter, isMaterializedView, defs)); } return tables; http://git-wip-us.apache.org/repos/asf/cassandra/blob/3bdcaa33/test/long/org/apache/cassandra/cql3/MaterializedViewLongTest.java ---------------------------------------------------------------------- diff --git a/test/long/org/apache/cassandra/cql3/MaterializedViewLongTest.java b/test/long/org/apache/cassandra/cql3/MaterializedViewLongTest.java new file mode 100644 index 0000000..70ec451 --- /dev/null +++ b/test/long/org/apache/cassandra/cql3/MaterializedViewLongTest.java @@ -0,0 +1,194 @@ +/* + * 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.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CyclicBarrier; + +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import com.datastax.driver.core.Row; +import com.datastax.driver.core.exceptions.NoHostAvailableException; +import com.datastax.driver.core.exceptions.WriteTimeoutException; +import org.apache.cassandra.concurrent.SEPExecutor; +import org.apache.cassandra.concurrent.Stage; +import org.apache.cassandra.concurrent.StageManager; +import org.apache.cassandra.db.BatchlogManager; +import org.apache.cassandra.utils.WrappedRunnable; + +public class MaterializedViewLongTest extends CQLTester +{ + int protocolVersion = 4; + private final List<String> materializedViews = new ArrayList<>(); + + @BeforeClass + public static void startup() + { + requireNetwork(); + } + @Before + public void begin() + { + materializedViews.clear(); + } + + @After + public void end() throws Throwable + { + for (String viewName : materializedViews) + executeNet(protocolVersion, "DROP MATERIALIZED VIEW " + viewName); + } + + private void createView(String name, String query) throws Throwable + { + executeNet(protocolVersion, String.format(query, name)); + // If exception is thrown, the view will not be added to the list; since it shouldn't have been created, this is + // the desired behavior + materializedViews.add(name); + } + + @Test + public void testConflictResolution() throws Throwable + { + final int writers = 96; + final int insertsPerWriter = 50; + final Map<Integer, Exception> failedWrites = new ConcurrentHashMap<>(); + + createTable("CREATE TABLE %s (" + + "a int," + + "b int," + + "c int," + + "PRIMARY KEY (a, b))"); + + executeNet(protocolVersion, "USE " + keyspace()); + + createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE c IS NOT NULL AND a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (c, a, b)"); + + CyclicBarrier semaphore = new CyclicBarrier(writers); + + Thread[] threads = new Thread[writers]; + for (int i = 0; i < writers; i++) + { + final int writer = i; + Thread t = new Thread(new WrappedRunnable() + { + public void runMayThrow() + { + try + { + int writerOffset = writer * insertsPerWriter; + semaphore.await(); + for (int i = 0; i < insertsPerWriter; i++) + { + try + { + executeNet(protocolVersion, "INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", + 1, + 1, + i + writerOffset); + } + catch (NoHostAvailableException|WriteTimeoutException e) + { + failedWrites.put(i + writerOffset, e); + } + } + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + } + }); + t.start(); + threads[i] = t; + } + + for (int i = 0; i < writers; i++) + threads[i].join(); + + for (int i = 0; i < writers * insertsPerWriter; i++) + { + if (executeNet(protocolVersion, "SELECT COUNT(*) FROM system.batchlog").one().getLong(0) == 0) + break; + try + { + // This will throw exceptions whenever there are exceptions trying to push the materialized view values + // out, caused by the view becoming overwhelmed. + BatchlogManager.instance.startBatchlogReplay().get(); + } + catch (Throwable ignore) + { + + } + } + + while (!(((SEPExecutor) StageManager.getStage(Stage.MATERIALIZED_VIEW_MUTATION)).getPendingTasks() == 0 + && ((SEPExecutor) StageManager.getStage(Stage.MATERIALIZED_VIEW_MUTATION)).getActiveCount() == 0)) + { + Thread.sleep(1); + } + + int value = executeNet(protocolVersion, "SELECT c FROM %s WHERE a = 1 AND b = 1").one().getInt("c"); + + List<Row> rows = executeNet(protocolVersion, "SELECT c FROM " + keyspace() + ".mv").all(); + + boolean containsC = false; + StringBuilder others = new StringBuilder(); + StringBuilder overlappingFailedWrites = new StringBuilder(); + for (Row row : rows) + { + int c = row.getInt("c"); + if (c == value) + containsC = true; + else + { + if (others.length() != 0) + others.append(' '); + others.append(c); + if (failedWrites.containsKey(c)) + { + if (overlappingFailedWrites.length() != 0) + overlappingFailedWrites.append(' '); + overlappingFailedWrites.append(c) + .append(':') + .append(failedWrites.get(c).getMessage()); + } + } + } + + if (rows.size() > 1) + { + throw new AssertionError(String.format("Expected 1 row, but found %d; %s c = %d, and (%s) of which (%s) failed to insert", rows.size(), containsC ? "found row with" : "no rows contained", value, others, overlappingFailedWrites)); + } + else if (rows.isEmpty()) + { + throw new AssertionError(String.format("Could not find row with c = %d", value)); + } + else if (rows.size() == 1 && !containsC) + { + throw new AssertionError(String.format("Single row had c = %d, expected %d", rows.get(0).getInt("c"), value)); + } + } +} http://git-wip-us.apache.org/repos/asf/cassandra/blob/3bdcaa33/test/unit/org/apache/cassandra/AbstractReadCommandBuilder.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/AbstractReadCommandBuilder.java b/test/unit/org/apache/cassandra/AbstractReadCommandBuilder.java deleted file mode 100644 index 575036f..0000000 --- a/test/unit/org/apache/cassandra/AbstractReadCommandBuilder.java +++ /dev/null @@ -1,327 +0,0 @@ -/* - * - * 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; - -import java.nio.ByteBuffer; -import java.util.*; - -import org.apache.cassandra.config.ColumnDefinition; -import org.apache.cassandra.cql3.*; -import org.apache.cassandra.db.*; -import org.apache.cassandra.db.rows.Row; -import org.apache.cassandra.db.filter.*; -import org.apache.cassandra.db.partitions.*; -import org.apache.cassandra.db.marshal.*; -import org.apache.cassandra.dht.*; -import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.utils.FBUtilities; - -public abstract class AbstractReadCommandBuilder -{ - protected final ColumnFamilyStore cfs; - protected int nowInSeconds; - - private int cqlLimit = -1; - private int pagingLimit = -1; - private boolean reversed = false; - - private Set<ColumnIdentifier> columns; - protected final RowFilter filter = RowFilter.create(); - - private Slice.Bound lowerClusteringBound; - private Slice.Bound upperClusteringBound; - - private NavigableSet<Clustering> clusterings; - - // Use Util.cmd() instead of this ctor directly - AbstractReadCommandBuilder(ColumnFamilyStore cfs) - { - this.cfs = cfs; - this.nowInSeconds = FBUtilities.nowInSeconds(); - } - - public AbstractReadCommandBuilder withNowInSeconds(int nowInSec) - { - this.nowInSeconds = nowInSec; - return this; - } - - public AbstractReadCommandBuilder fromIncl(Object... values) - { - assert lowerClusteringBound == null && clusterings == null; - this.lowerClusteringBound = Slice.Bound.create(cfs.metadata.comparator, true, true, values); - return this; - } - - public AbstractReadCommandBuilder fromExcl(Object... values) - { - assert lowerClusteringBound == null && clusterings == null; - this.lowerClusteringBound = Slice.Bound.create(cfs.metadata.comparator, true, false, values); - return this; - } - - public AbstractReadCommandBuilder toIncl(Object... values) - { - assert upperClusteringBound == null && clusterings == null; - this.upperClusteringBound = Slice.Bound.create(cfs.metadata.comparator, false, true, values); - return this; - } - - public AbstractReadCommandBuilder toExcl(Object... values) - { - assert upperClusteringBound == null && clusterings == null; - this.upperClusteringBound = Slice.Bound.create(cfs.metadata.comparator, false, false, values); - return this; - } - - public AbstractReadCommandBuilder includeRow(Object... values) - { - assert lowerClusteringBound == null && upperClusteringBound == null; - - if (this.clusterings == null) - this.clusterings = new TreeSet<>(cfs.metadata.comparator); - - this.clusterings.add(cfs.metadata.comparator.make(values)); - return this; - } - - public AbstractReadCommandBuilder reverse() - { - this.reversed = true; - return this; - } - - public AbstractReadCommandBuilder withLimit(int newLimit) - { - this.cqlLimit = newLimit; - return this; - } - - public AbstractReadCommandBuilder withPagingLimit(int newLimit) - { - this.pagingLimit = newLimit; - return this; - } - - public AbstractReadCommandBuilder columns(String... columns) - { - if (this.columns == null) - this.columns = new HashSet<>(); - - for (String column : columns) - this.columns.add(ColumnIdentifier.getInterned(column, true)); - return this; - } - - private ByteBuffer bb(Object value, AbstractType<?> type) - { - return value instanceof ByteBuffer ? (ByteBuffer)value : ((AbstractType)type).decompose(value); - } - - private AbstractType<?> forValues(AbstractType<?> collectionType) - { - assert collectionType instanceof CollectionType; - CollectionType ct = (CollectionType)collectionType; - switch (ct.kind) - { - case LIST: - case MAP: - return ct.valueComparator(); - case SET: - return ct.nameComparator(); - } - throw new AssertionError(); - } - - private AbstractType<?> forKeys(AbstractType<?> collectionType) - { - assert collectionType instanceof CollectionType; - CollectionType ct = (CollectionType)collectionType; - switch (ct.kind) - { - case LIST: - case MAP: - return ct.nameComparator(); - } - throw new AssertionError(); - } - - public AbstractReadCommandBuilder filterOn(String column, Operator op, Object value) - { - ColumnDefinition def = cfs.metadata.getColumnDefinition(ColumnIdentifier.getInterned(column, true)); - assert def != null; - - AbstractType<?> type = def.type; - if (op == Operator.CONTAINS) - type = forValues(type); - else if (op == Operator.CONTAINS_KEY) - type = forKeys(type); - - this.filter.add(def, op, bb(value, type)); - return this; - } - - protected ColumnFilter makeColumnFilter() - { - if (columns == null) - return ColumnFilter.all(cfs.metadata); - - ColumnFilter.Builder builder = ColumnFilter.allColumnsBuilder(cfs.metadata); - for (ColumnIdentifier column : columns) - builder.add(cfs.metadata.getColumnDefinition(column)); - return builder.build(); - } - - protected ClusteringIndexFilter makeFilter() - { - if (clusterings != null) - { - return new ClusteringIndexNamesFilter(clusterings, reversed); - } - else - { - Slice slice = Slice.make(lowerClusteringBound == null ? Slice.Bound.BOTTOM : lowerClusteringBound, - upperClusteringBound == null ? Slice.Bound.TOP : upperClusteringBound); - return new ClusteringIndexSliceFilter(Slices.with(cfs.metadata.comparator, slice), reversed); - } - } - - protected DataLimits makeLimits() - { - DataLimits limits = cqlLimit < 0 ? DataLimits.NONE : DataLimits.cqlLimits(cqlLimit); - if (pagingLimit >= 0) - limits = limits.forPaging(pagingLimit); - return limits; - } - - public Row getOnlyRow() - { - return Util.getOnlyRow(build()); - } - - public Row getOnlyRowUnfiltered() - { - return Util.getOnlyRowUnfiltered(build()); - } - - public FilteredPartition getOnlyPartition() - { - return Util.getOnlyPartition(build()); - } - - public Partition getOnlyPartitionUnfiltered() - { - return Util.getOnlyPartitionUnfiltered(build()); - } - - public abstract ReadCommand build(); - - public static class SinglePartitionBuilder extends AbstractReadCommandBuilder - { - private final DecoratedKey partitionKey; - - SinglePartitionBuilder(ColumnFamilyStore cfs, DecoratedKey key) - { - super(cfs); - this.partitionKey = key; - } - - @Override - public ReadCommand build() - { - return SinglePartitionReadCommand.create(cfs.metadata, nowInSeconds, makeColumnFilter(), filter, makeLimits(), partitionKey, makeFilter()); - } - } - - public static class PartitionRangeBuilder extends AbstractReadCommandBuilder - { - private DecoratedKey startKey; - private boolean startInclusive; - private DecoratedKey endKey; - private boolean endInclusive; - - PartitionRangeBuilder(ColumnFamilyStore cfs) - { - super(cfs); - } - - public PartitionRangeBuilder fromKeyIncl(Object... values) - { - assert startKey == null; - this.startInclusive = true; - this.startKey = Util.makeKey(cfs.metadata, values); - return this; - } - - public PartitionRangeBuilder fromKeyExcl(Object... values) - { - assert startKey == null; - this.startInclusive = false; - this.startKey = Util.makeKey(cfs.metadata, values); - return this; - } - - public PartitionRangeBuilder toKeyIncl(Object... values) - { - assert endKey == null; - this.endInclusive = true; - this.endKey = Util.makeKey(cfs.metadata, values); - return this; - } - - public PartitionRangeBuilder toKeyExcl(Object... values) - { - assert endKey == null; - this.endInclusive = false; - this.endKey = Util.makeKey(cfs.metadata, values); - return this; - } - - @Override - public ReadCommand build() - { - PartitionPosition start = startKey; - if (start == null) - { - start = StorageService.getPartitioner().getMinimumToken().maxKeyBound(); - startInclusive = false; - } - PartitionPosition end = endKey; - if (end == null) - { - end = StorageService.getPartitioner().getMinimumToken().maxKeyBound(); - endInclusive = true; - } - - AbstractBounds<PartitionPosition> bounds; - if (startInclusive && endInclusive) - bounds = new Bounds<PartitionPosition>(start, end); - else if (startInclusive && !endInclusive) - bounds = new IncludingExcludingBounds<PartitionPosition>(start, end); - else if (!startInclusive && endInclusive) - bounds = new Range<PartitionPosition>(start, end); - else - bounds = new ExcludingBounds<PartitionPosition>(start, end); - - return new PartitionRangeReadCommand(cfs.metadata, nowInSeconds, makeColumnFilter(), filter, makeLimits(), new DataRange(bounds, makeFilter())); - } - } -} http://git-wip-us.apache.org/repos/asf/cassandra/blob/3bdcaa33/test/unit/org/apache/cassandra/cql3/CQLTester.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index ddc41c7..458d7dd 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -760,8 +760,8 @@ public abstract class CQLTester if (!Objects.equal(expectedByteValue, actualValue)) { - Object actualValueDecoded = column.type.getSerializer().deserialize(actualValue); - if (!actualValueDecoded.equals(expected[j])) + Object actualValueDecoded = actualValue == null ? null : column.type.getSerializer().deserialize(actualValue); + if (!expected[j].equals(actualValueDecoded)) Assert.fail(String.format("Invalid value for row %d column %d (%s of type %s), expected <%s> but got <%s>", i, j, http://git-wip-us.apache.org/repos/asf/cassandra/blob/3bdcaa33/test/unit/org/apache/cassandra/cql3/MaterializedViewTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/cql3/MaterializedViewTest.java b/test/unit/org/apache/cassandra/cql3/MaterializedViewTest.java new file mode 100644 index 0000000..78c40e2 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/MaterializedViewTest.java @@ -0,0 +1,955 @@ +/* + * 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.math.BigDecimal; +import java.math.BigInteger; +import java.net.InetAddress; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashSet; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import com.datastax.driver.core.*; +import com.datastax.driver.core.exceptions.InvalidQueryException; +import junit.framework.Assert; +import org.apache.cassandra.concurrent.SEPExecutor; +import org.apache.cassandra.concurrent.Stage; +import org.apache.cassandra.concurrent.StageManager; +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.config.Schema; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.SystemKeyspace; +import org.apache.cassandra.serializers.SimpleDateSerializer; +import org.apache.cassandra.serializers.TimeSerializer; +import org.apache.cassandra.utils.ByteBufferUtil; + +public class MaterializedViewTest extends CQLTester +{ + int protocolVersion = 4; + private final List<String> materializedViews = new ArrayList<>(); + + @BeforeClass + public static void startup() + { + requireNetwork(); + } + @Before + public void begin() + { + materializedViews.clear(); + } + + @After + public void end() throws Throwable + { + for (String viewName : materializedViews) + executeNet(protocolVersion, "DROP MATERIALIZED VIEW " + viewName); + } + + private void createView(String name, String query) throws Throwable + { + executeNet(protocolVersion, String.format(query, name)); + // If exception is thrown, the view will not be added to the list; since it shouldn't have been created, this is + // the desired behavior + materializedViews.add(name); + } + + private void updateMV(String query, Object... params) throws Throwable + { + executeNet(protocolVersion, query, params); + while (!(((SEPExecutor) StageManager.getStage(Stage.MATERIALIZED_VIEW_MUTATION)).getPendingTasks() == 0 + && ((SEPExecutor) StageManager.getStage(Stage.MATERIALIZED_VIEW_MUTATION)).getActiveCount() == 0)) + { + Thread.sleep(1); + } + } + + + @Test + public void testAccessAndSchema() throws Throwable + { + createTable("CREATE TABLE %s (" + + "k int, " + + "asciival ascii, " + + "bigintval bigint, " + + "PRIMARY KEY((k, asciival)))"); + + execute("USE " + keyspace()); + executeNet(protocolVersion, "USE " + keyspace()); + + createView("mv1_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE bigintval IS NOT NULL AND k IS NOT NULL AND asciival IS NOT NULL PRIMARY KEY (bigintval, k, asciival)"); + updateMV("INSERT INTO %s(k,asciival,bigintval)VALUES(?,?,?)", 0, "foo", 1L); + + try + { + updateMV("INSERT INTO mv1_test(k,asciival,bigintval) VALUES(?,?,?)", 1, "foo", 2L); + Assert.fail("Shouldn't be able to modify a MV directly"); + } + catch (Exception e) + { + } + + try + { + executeNet(protocolVersion, "ALTER TABLE mv1_test ADD foo text"); + Assert.fail("Should not be able to use alter table with MV"); + } + catch (Exception e) + { + } + + try + { + executeNet(protocolVersion, "ALTER TABLE mv1_test WITH compaction = { 'class' : 'LeveledCompactionStrategy' }"); + Assert.fail("Should not be able to use alter table with MV"); + } + catch (Exception e) + { + } + + executeNet(protocolVersion, "ALTER MATERIALIZED VIEW mv1_test WITH compaction = { 'class' : 'LeveledCompactionStrategy' }"); + + //Test alter add + executeNet(protocolVersion, "ALTER TABLE %s ADD foo text"); + CFMetaData metadata = Schema.instance.getCFMetaData(keyspace(), "mv1_test"); + Assert.assertNotNull(metadata.getColumnDefinition(ByteBufferUtil.bytes("foo"))); + + updateMV("INSERT INTO %s(k,asciival,bigintval,foo)VALUES(?,?,?,?)", 0, "foo", 1L, "bar"); + assertRows(execute("SELECT foo from %s"), row("bar")); + + //Test alter rename + executeNet(protocolVersion, "ALTER TABLE %s RENAME asciival TO bar"); + + assertRows(execute("SELECT bar from %s"), row("foo")); + metadata = Schema.instance.getCFMetaData(keyspace(), "mv1_test"); + Assert.assertNotNull(metadata.getColumnDefinition(ByteBufferUtil.bytes("bar"))); + } + + + @Test + public void testStaticTable() throws Throwable + { + createTable("CREATE TABLE %s (" + + "k int, " + + "c int, " + + "sval text static, " + + "val text, " + + "PRIMARY KEY(k,c))"); + + execute("USE " + keyspace()); + executeNet(protocolVersion, "USE " + keyspace()); + + try + { + createView("mv_static", "CREATE MATERIALIZED VIEW %%s AS SELECT * FROM %s WHERE sval IS NOT NULL AND k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (sval,k,c)"); + Assert.fail("MV on static should fail"); + } + catch (InvalidQueryException e) + { + } + + createView("mv_static", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE val IS NOT NULL AND k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (val,k,c)"); + + for (int i = 0; i < 100; i++) + updateMV("INSERT into %s (k,c,sval,val)VALUES(?,?,?,?)", 0, i % 2, "bar" + i, "baz"); + + Assert.assertEquals(2, execute("select * from %s").size()); + + assertRows(execute("SELECT sval from %s"), row("bar99"), row("bar99")); + + Assert.assertEquals(2, execute("select * from mv_static").size()); + + assertInvalid("SELECT sval from mv_static"); + } + + + @Test + public void testOldTimestamps() throws Throwable + { + createTable("CREATE TABLE %s (" + + "k int, " + + "c int, " + + "val text, " + + "PRIMARY KEY(k,c))"); + + execute("USE " + keyspace()); + executeNet(protocolVersion, "USE " + keyspace()); + + createView("mv_tstest", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE val IS NOT NULL AND k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (val,k,c)"); + + for (int i = 0; i < 100; i++) + updateMV("INSERT into %s (k,c,val)VALUES(?,?,?)", 0, i % 2, "baz"); + + Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()).forceBlockingFlush(); + + Assert.assertEquals(2, execute("select * from %s").size()); + Assert.assertEquals(2, execute("select * from mv_tstest").size()); + + assertRows(execute("SELECT val from %s where k = 0 and c = 0"), row("baz")); + assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "baz"), row(0), row(1)); + + //Make sure an old TS does nothing + updateMV("UPDATE %s USING TIMESTAMP 100 SET val = ? where k = ? AND c = ?", "bar", 0, 0); + assertRows(execute("SELECT val from %s where k = 0 and c = 0"), row("baz")); + assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "baz"), row(0), row(1)); + assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "bar")); + + //Latest TS + updateMV("UPDATE %s SET val = ? where k = ? AND c = ?", "bar", 0, 0); + assertRows(execute("SELECT val from %s where k = 0 and c = 0"), row("bar")); + assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "bar"), row(0)); + assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "baz"), row(1)); + } + + @Test + public void testCountersTable() throws Throwable + { + createTable("CREATE TABLE %s (" + + "k int PRIMARY KEY, " + + "count counter)"); + + execute("USE " + keyspace()); + executeNet(protocolVersion, "USE " + keyspace()); + + try + { + createView("mv_counter", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE count IS NOT NULL AND k IS NOT NULL PRIMARY KEY (count,k)"); + Assert.fail("MV on counter should fail"); + } + catch (InvalidQueryException e) + { + } + } + + @Test + public void testRangeTombstone() throws Throwable + { + createTable("CREATE TABLE %s (" + + "k int, " + + "asciival ascii, " + + "bigintval bigint, " + + "textval1 text, " + + "textval2 text, " + + "PRIMARY KEY((k, asciival), bigintval, textval1)" + + ")"); + + execute("USE " + keyspace()); + executeNet(protocolVersion, "USE " + keyspace()); + + createView("mv_test1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE textval2 IS NOT NULL AND k IS NOT NULL AND asciival IS NOT NULL AND bigintval IS NOT NULL AND textval1 IS NOT NULL PRIMARY KEY ((textval2, k), asciival, bigintval, textval1)"); + + for (int i = 0; i < 100; i++) + updateMV("INSERT into %s (k,asciival,bigintval,textval1,textval2)VALUES(?,?,?,?,?)", 0, "foo", (long) i % 2, "bar" + i, "baz"); + + Assert.assertEquals(50, execute("select * from %s where k = 0 and asciival = 'foo' and bigintval = 0").size()); + Assert.assertEquals(50, execute("select * from %s where k = 0 and asciival = 'foo' and bigintval = 1").size()); + + Assert.assertEquals(100, execute("select * from mv_test1").size()); + + //Check the builder works + createView("mv_test2", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE textval2 IS NOT NULL AND k IS NOT NULL AND asciival IS NOT NULL AND bigintval IS NOT NULL AND textval1 IS NOT NULL PRIMARY KEY ((textval2, k), asciival, bigintval, textval1)"); + + while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test2")) + Thread.sleep(1000); + + Assert.assertEquals(100, execute("select * from mv_test2").size()); + + createView("mv_test3", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE textval2 IS NOT NULL AND k IS NOT NULL AND asciival IS NOT NULL AND bigintval IS NOT NULL AND textval1 IS NOT NULL PRIMARY KEY ((textval2, k), bigintval, textval1, asciival)"); + + while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test3")) + Thread.sleep(1000); + + Assert.assertEquals(100, execute("select * from mv_test3").size()); + Assert.assertEquals(100, execute("select asciival from mv_test3 where textval2 = ? and k = ?", "baz", 0).size()); + + //Write a RT and verify the data is removed from index + updateMV("DELETE FROM %s WHERE k = ? AND asciival = ? and bigintval = ?", 0, "foo", 0L); + + Assert.assertEquals(50, execute("select asciival from mv_test3 where textval2 = ? and k = ?", "baz", 0).size()); + } + + + @Test + public void testRangeTombstone2() throws Throwable + { + createTable("CREATE TABLE %s (" + + "k int, " + + "asciival ascii, " + + "bigintval bigint, " + + "textval1 text, " + + "PRIMARY KEY((k, asciival), bigintval)" + + ")"); + + execute("USE " + keyspace()); + executeNet(protocolVersion, "USE " + keyspace()); + + createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE textval1 IS NOT NULL AND k IS NOT NULL AND asciival IS NOT NULL AND bigintval IS NOT NULL PRIMARY KEY ((textval1, k), asciival, bigintval)"); + + for (int i = 0; i < 100; i++) + updateMV("INSERT into %s (k,asciival,bigintval,textval1)VALUES(?,?,?,?)", 0, "foo", (long) i % 2, "bar" + i); + + Assert.assertEquals(1, execute("select * from %s where k = 0 and asciival = 'foo' and bigintval = 0").size()); + Assert.assertEquals(1, execute("select * from %s where k = 0 and asciival = 'foo' and bigintval = 1").size()); + + + Assert.assertEquals(2, execute("select * from %s").size()); + Assert.assertEquals(2, execute("select * from mv").size()); + + //Write a RT and verify the data is removed from index + updateMV("DELETE FROM %s WHERE k = ? AND asciival = ? and bigintval = ?", 0, "foo", 0L); + + Assert.assertEquals(1, execute("select * from %s").size()); + Assert.assertEquals(1, execute("select * from mv").size()); + } + + @Test + public void testCompoundPartitionKey() throws Throwable + { + createTable("CREATE TABLE %s (" + + "k int, " + + "asciival ascii, " + + "bigintval bigint, " + + "PRIMARY KEY((k, asciival)))"); + + CFMetaData metadata = currentTableMetadata(); + + execute("USE " + keyspace()); + executeNet(protocolVersion, "USE " + keyspace()); + + for (ColumnDefinition def : new HashSet<>(metadata.allColumns())) + { + try + { + String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL " + + (def.name.toString().equals("asciival") ? "" : "AND asciival IS NOT NULL ") + "PRIMARY KEY (" + + def.name + ", k" + (def.name.toString().equals("asciival") ? "" : ", asciival") + ")"; + createView("mv1_" + def.name, query); + + if (def.type.isMultiCell()) + Assert.fail("MV on a multicell should fail " + def); + } + catch (InvalidQueryException e) + { + if (!def.type.isMultiCell() && !def.isPartitionKey()) + Assert.fail("MV creation failed on " + def); + } + + + try + { + String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL " + + (def.name.toString().equals("asciival") ? "" : "AND asciival IS NOT NULL ") + " PRIMARY KEY (" + + def.name + ", asciival" + (def.name.toString().equals("k") ? "" : ", k") + ")"; + createView("mv2_" + def.name, query); + + if (def.type.isMultiCell()) + Assert.fail("MV on a multicell should fail " + def); + } + catch (InvalidQueryException e) + { + if (!def.type.isMultiCell() && !def.isPartitionKey()) + Assert.fail("MV creation failed on " + def); + } + + try + { + String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL " + + (def.name.toString().equals("asciival") ? "" : "AND asciival IS NOT NULL ") + "PRIMARY KEY ((" + def.name + ", k), asciival)"; + createView("mv3_" + def.name, query); + + if (def.type.isMultiCell()) + Assert.fail("MV on a multicell should fail " + def); + } + catch (InvalidQueryException e) + { + if (!def.type.isMultiCell() && !def.isPartitionKey()) + Assert.fail("MV creation failed on " + def); + } + + + try + { + String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL " + + (def.name.toString().equals("asciival") ? "" : "AND asciival IS NOT NULL ") + "PRIMARY KEY ((" + def.name + ", k), asciival)"; + createView("mv3_" + def.name, query); + + Assert.fail("Should fail on duplicate name"); + } + catch (Exception e) + { + } + + try + { + String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL " + + (def.name.toString().equals("asciival") ? "" : "AND asciival IS NOT NULL ") + "PRIMARY KEY ((" + def.name + ", k), nonexistentcolumn)"; + createView("mv3_" + def.name, query); + Assert.fail("Should fail with unknown base column"); + } + catch (InvalidQueryException e) + { + } + } + + updateMV("INSERT INTO %s (k, asciival, bigintval) VALUES (?, ?, fromJson(?))", 0, "ascii text", "123123123123"); + updateMV("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"ascii text\""); + assertRows(execute("SELECT bigintval FROM %s WHERE k = ? and asciival = ?", 0, "ascii text"), row(123123123123L)); + + //Check the MV + assertRows(execute("SELECT k, bigintval from mv1_asciival WHERE asciival = ?", "ascii text"), row(0, 123123123123L)); + assertRows(execute("SELECT k, bigintval from mv2_k WHERE asciival = ? and k = ?", "ascii text", 0), row(0, 123123123123L)); + assertRows(execute("SELECT k from mv1_bigintval WHERE bigintval = ?", 123123123123L), row(0)); + assertRows(execute("SELECT asciival from mv3_bigintval where bigintval = ? AND k = ?", 123123123123L, 0), row("ascii text")); + + + //UPDATE BASE + updateMV("INSERT INTO %s (k, asciival, bigintval) VALUES (?, ?, fromJson(?))", 0, "ascii text", "1"); + assertRows(execute("SELECT bigintval FROM %s WHERE k = ? and asciival = ?", 0, "ascii text"), row(1L)); + + //Check the MV + assertRows(execute("SELECT k, bigintval from mv1_asciival WHERE asciival = ?", "ascii text"), row(0, 1L)); + assertRows(execute("SELECT k, bigintval from mv2_k WHERE asciival = ? and k = ?", "ascii text", 0), row(0, 1L)); + assertRows(execute("SELECT k from mv1_bigintval WHERE bigintval = ?", 123123123123L)); + assertRows(execute("SELECT asciival from mv3_bigintval where bigintval = ? AND k = ?", 123123123123L, 0)); + assertRows(execute("SELECT asciival from mv3_bigintval where bigintval = ? AND k = ?", 1L, 0), row("ascii text")); + + + //test truncate also truncates all MV + updateMV("TRUNCATE %s"); + + assertRows(execute("SELECT bigintval FROM %s WHERE k = ? and asciival = ?", 0, "ascii text")); + assertRows(execute("SELECT k, bigintval from mv1_asciival WHERE asciival = ?", "ascii text")); + assertRows(execute("SELECT k, bigintval from mv2_k WHERE asciival = ? and k = ?", "ascii text", 0)); + assertRows(execute("SELECT asciival from mv3_bigintval where bigintval = ? AND k = ?", 1L, 0)); + } + + @Test + public void testCollections() throws Throwable + { + createTable("CREATE TABLE %s (" + + "k int, " + + "intval int, " + + "listval list<int>, " + + "PRIMARY KEY (k))"); + + execute("USE " + keyspace()); + executeNet(protocolVersion, "USE " + keyspace()); + + createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND intval IS NOT NULL PRIMARY KEY (intval, k)"); + + updateMV("INSERT INTO %s (k, intval, listval) VALUES (?, ?, fromJson(?))", 0, 0, "[1, 2, 3]"); + assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(1, 2, 3))); + assertRows(execute("SELECT k, listval from mv WHERE intval = ?", 0), row(0, list(1, 2, 3))); + + updateMV("INSERT INTO %s (k, intval) VALUES (?, ?)", 1, 1); + updateMV("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 1, "[1, 2, 3]"); + assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 1), row(1, list(1, 2, 3))); + assertRows(execute("SELECT k, listval from mv WHERE intval = ?", 1), row(1, list(1, 2, 3))); + } + + @Test + public void testUpdate() throws Throwable + { + createTable("CREATE TABLE %s (" + + "k int, " + + "intval int, " + + "PRIMARY KEY (k))"); + + execute("USE " + keyspace()); + executeNet(protocolVersion, "USE " + keyspace()); + + createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND intval IS NOT NULL PRIMARY KEY (intval, k)"); + + updateMV("INSERT INTO %s (k, intval) VALUES (?, ?)", 0, 0); + assertRows(execute("SELECT k, intval FROM %s WHERE k = ?", 0), row(0, 0)); + assertRows(execute("SELECT k, intval from mv WHERE intval = ?", 0), row(0, 0)); + + updateMV("INSERT INTO %s (k, intval) VALUES (?, ?)", 0, 1); + assertRows(execute("SELECT k, intval FROM %s WHERE k = ?", 0), row(0, 1)); + assertRows(execute("SELECT k, intval from mv WHERE intval = ?", 1), row(0, 1)); + } + + @Test + public void testDecimalUpdate() throws Throwable + { + createTable("CREATE TABLE %s (" + + "k int, " + + "decimalval decimal, " + + "asciival ascii, " + + "PRIMARY KEY (k))"); + + execute("USE " + keyspace()); + executeNet(protocolVersion, "USE " + keyspace()); + + createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND decimalval IS NOT NULL PRIMARY KEY (decimalval, k)"); + + updateMV("INSERT INTO %s (k, asciival) VALUES (?, ?)", 0, "ascii text"); + updateMV("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "123123"); + assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123"))); + assertRows(execute("SELECT k, asciival from mv WHERE decimalval = fromJson(?)", "123123.123123")); + assertRows(execute("SELECT k, asciival from mv WHERE decimalval = fromJson(?)", "123123"), row(0, "ascii text")); + + // accept strings for numbers that cannot be represented as doubles + updateMV("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "\"123123.123123\""); + assertRows(execute("SELECT k, decimalval, asciival FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123.123123"), "ascii text")); + assertRows(execute("SELECT k, asciival from mv WHERE decimalval = fromJson(?)", "\"123123.123123\""), row(0, "ascii text")); + } + + @Test + public void testAllTypes() throws Throwable + { + String myType = createType("CREATE TYPE %s (a int, b uuid, c set<text>)"); + + createTable("CREATE TABLE %s (" + + "k int PRIMARY KEY, " + + "asciival ascii, " + + "bigintval bigint, " + + "blobval blob, " + + "booleanval boolean, " + + "dateval date, " + + "decimalval decimal, " + + "doubleval double, " + + "floatval float, " + + "inetval inet, " + + "intval int, " + + "textval text, " + + "timeval time, " + + "timestampval timestamp, " + + "timeuuidval timeuuid, " + + "uuidval uuid," + + "varcharval varchar, " + + "varintval varint, " + + "listval list<int>, " + + "frozenlistval frozen<list<int>>, " + + "setval set<uuid>, " + + "frozensetval frozen<set<uuid>>, " + + "mapval map<ascii, int>," + + "frozenmapval frozen<map<ascii, int>>," + + "tupleval frozen<tuple<int, ascii, uuid>>," + + "udtval frozen<" + myType + ">)"); + + CFMetaData metadata = currentTableMetadata(); + + execute("USE " + keyspace()); + executeNet(protocolVersion, "USE " + keyspace()); + + for (ColumnDefinition def : new HashSet<>(metadata.allColumns())) + { + try + { + createView("mv_" + def.name, "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL PRIMARY KEY (" + def.name + ",k)"); + + if (def.type.isMultiCell()) + Assert.fail("MV on a multicell should fail " + def); + + if (def.isPartitionKey()) + Assert.fail("MV on partition key should fail " + def); + } + catch (InvalidQueryException e) + { + if (!def.type.isMultiCell() && !def.isPartitionKey()) + Assert.fail("MV creation failed on " + def); + } + } + + // fromJson() can only be used when the receiver type is known + assertInvalidMessage("fromJson() cannot be used in the selection clause", "SELECT fromJson(asciival) FROM %s", 0, 0); + + String func1 = createFunction(KEYSPACE, "int", "CREATE FUNCTION %s (a int) CALLED ON NULL INPUT RETURNS text LANGUAGE java AS $$ return a.toString(); $$"); + createFunctionOverload(func1, "int", "CREATE FUNCTION %s (a text) CALLED ON NULL INPUT RETURNS text LANGUAGE java AS $$ return new String(a); $$"); + + // ================ ascii ================ + updateMV("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"ascii text\""); + assertRows(execute("SELECT k, asciival FROM %s WHERE k = ?", 0), row(0, "ascii text")); + + updateMV("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"ascii \\\" text\""); + assertRows(execute("SELECT k, asciival FROM %s WHERE k = ?", 0), row(0, "ascii \" text")); + + // test that we can use fromJson() in other valid places in queries + assertRows(execute("SELECT asciival FROM %s WHERE k = fromJson(?)", "0"), row("ascii \" text")); + + //Check the MV + assertRows(execute("SELECT k, udtval from mv_asciival WHERE asciival = ?", "ascii text")); + assertRows(execute("SELECT k, udtval from mv_asciival WHERE asciival = ?", "ascii \" text"), row(0, null)); + + updateMV("UPDATE %s SET asciival = fromJson(?) WHERE k = fromJson(?)", "\"ascii \\\" text\"", "0"); + assertRows(execute("SELECT k, udtval from mv_asciival WHERE asciival = ?", "ascii \" text"), row(0, null)); + + updateMV("DELETE FROM %s WHERE k = fromJson(?)", "0"); + assertRows(execute("SELECT k, asciival FROM %s WHERE k = ?", 0)); + assertRows(execute("SELECT k, udtval from mv_asciival WHERE asciival = ?", "ascii \" text")); + + updateMV("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"ascii text\""); + assertRows(execute("SELECT k, udtval from mv_asciival WHERE asciival = ?", "ascii text"), row(0, null)); + + // ================ bigint ================ + updateMV("INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "123123123123"); + assertRows(execute("SELECT k, bigintval FROM %s WHERE k = ?", 0), row(0, 123123123123L)); + assertRows(execute("SELECT k, asciival from mv_bigintval WHERE bigintval = ?", 123123123123L), row(0, "ascii text")); + + // ================ blob ================ + updateMV("INSERT INTO %s (k, blobval) VALUES (?, fromJson(?))", 0, "\"0x00000001\""); + assertRows(execute("SELECT k, blobval FROM %s WHERE k = ?", 0), row(0, ByteBufferUtil.bytes(1))); + assertRows(execute("SELECT k, asciival from mv_blobval WHERE blobval = ?", ByteBufferUtil.bytes(1)), row(0, "ascii text")); + + // ================ boolean ================ + updateMV("INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "true"); + assertRows(execute("SELECT k, booleanval FROM %s WHERE k = ?", 0), row(0, true)); + assertRows(execute("SELECT k, asciival from mv_booleanval WHERE booleanval = ?", true), row(0, "ascii text")); + + updateMV("INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "false"); + assertRows(execute("SELECT k, booleanval FROM %s WHERE k = ?", 0), row(0, false)); + assertRows(execute("SELECT k, asciival from mv_booleanval WHERE booleanval = ?", true)); + assertRows(execute("SELECT k, asciival from mv_booleanval WHERE booleanval = ?", false), row(0, "ascii text")); + + // ================ date ================ + updateMV("INSERT INTO %s (k, dateval) VALUES (?, fromJson(?))", 0, "\"1987-03-23\""); + assertRows(execute("SELECT k, dateval FROM %s WHERE k = ?", 0), row(0, SimpleDateSerializer.dateStringToDays("1987-03-23"))); + assertRows(execute("SELECT k, asciival from mv_dateval WHERE dateval = fromJson(?)", "\"1987-03-23\""), row(0, "ascii text")); + + // ================ decimal ================ + updateMV("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "123123.123123"); + assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123.123123"))); + assertRows(execute("SELECT k, asciival from mv_decimalval WHERE decimalval = fromJson(?)", "123123.123123"), row(0, "ascii text")); + + updateMV("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "123123"); + assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123"))); + assertRows(execute("SELECT k, asciival from mv_decimalval WHERE decimalval = fromJson(?)", "123123.123123")); + assertRows(execute("SELECT k, asciival from mv_decimalval WHERE decimalval = fromJson(?)", "123123"), row(0, "ascii text")); + + // accept strings for numbers that cannot be represented as doubles + updateMV("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "\"123123.123123\""); + assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123.123123"))); + + updateMV("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "\"-1.23E-12\""); + assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("-1.23E-12"))); + assertRows(execute("SELECT k, asciival from mv_decimalval WHERE decimalval = fromJson(?)", "\"-1.23E-12\""), row(0, "ascii text")); + + // ================ double ================ + updateMV("INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "123123.123123"); + assertRows(execute("SELECT k, doubleval FROM %s WHERE k = ?", 0), row(0, 123123.123123d)); + assertRows(execute("SELECT k, asciival from mv_doubleval WHERE doubleval = fromJson(?)", "123123.123123"), row(0, "ascii text")); + + updateMV("INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "123123"); + assertRows(execute("SELECT k, doubleval FROM %s WHERE k = ?", 0), row(0, 123123.0d)); + assertRows(execute("SELECT k, asciival from mv_doubleval WHERE doubleval = fromJson(?)", "123123"), row(0, "ascii text")); + + // ================ float ================ + updateMV("INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "123123.123123"); + assertRows(execute("SELECT k, floatval FROM %s WHERE k = ?", 0), row(0, 123123.123123f)); + assertRows(execute("SELECT k, asciival from mv_floatval WHERE floatval = fromJson(?)", "123123.123123"), row(0, "ascii text")); + + updateMV("INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "123123"); + assertRows(execute("SELECT k, floatval FROM %s WHERE k = ?", 0), row(0, 123123.0f)); + assertRows(execute("SELECT k, asciival from mv_floatval WHERE floatval = fromJson(?)", "123123"), row(0, "ascii text")); + + // ================ inet ================ + updateMV("INSERT INTO %s (k, inetval) VALUES (?, fromJson(?))", 0, "\"127.0.0.1\""); + assertRows(execute("SELECT k, inetval FROM %s WHERE k = ?", 0), row(0, InetAddress.getByName("127.0.0.1"))); + assertRows(execute("SELECT k, asciival from mv_inetval WHERE inetval = fromJson(?)", "\"127.0.0.1\""), row(0, "ascii text")); + + updateMV("INSERT INTO %s (k, inetval) VALUES (?, fromJson(?))", 0, "\"::1\""); + assertRows(execute("SELECT k, inetval FROM %s WHERE k = ?", 0), row(0, InetAddress.getByName("::1"))); + assertRows(execute("SELECT k, asciival from mv_inetval WHERE inetval = fromJson(?)", "\"127.0.0.1\"")); + assertRows(execute("SELECT k, asciival from mv_inetval WHERE inetval = fromJson(?)", "\"::1\""), row(0, "ascii text")); + + // ================ int ================ + updateMV("INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "123123"); + assertRows(execute("SELECT k, intval FROM %s WHERE k = ?", 0), row(0, 123123)); + assertRows(execute("SELECT k, asciival from mv_intval WHERE intval = fromJson(?)", "123123"), row(0, "ascii text")); + + // ================ text (varchar) ================ + updateMV("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"some \\\" text\""); + assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, "some \" text")); + + updateMV("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"\\u2013\""); + assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, "\u2013")); + assertRows(execute("SELECT k, asciival from mv_textval WHERE textval = fromJson(?)", "\"\\u2013\""), row(0, "ascii text")); + + updateMV("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"abcd\""); + assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, "abcd")); + assertRows(execute("SELECT k, asciival from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, "ascii text")); + + // ================ time ================ + updateMV("INSERT INTO %s (k, timeval) VALUES (?, fromJson(?))", 0, "\"07:35:07.000111222\""); + assertRows(execute("SELECT k, timeval FROM %s WHERE k = ?", 0), row(0, TimeSerializer.timeStringToLong("07:35:07.000111222"))); + assertRows(execute("SELECT k, asciival from mv_timeval WHERE timeval = fromJson(?)", "\"07:35:07.000111222\""), row(0, "ascii text")); + + // ================ timestamp ================ + updateMV("INSERT INTO %s (k, timestampval) VALUES (?, fromJson(?))", 0, "123123123123"); + assertRows(execute("SELECT k, timestampval FROM %s WHERE k = ?", 0), row(0, new Date(123123123123L))); + assertRows(execute("SELECT k, asciival from mv_timestampval WHERE timestampval = fromJson(?)", "123123123123"), row(0, "ascii text")); + + updateMV("INSERT INTO %s (k, timestampval) VALUES (?, fromJson(?))", 0, "\"2014-01-01\""); + assertRows(execute("SELECT k, timestampval FROM %s WHERE k = ?", 0), row(0, new SimpleDateFormat("y-M-d").parse("2014-01-01"))); + assertRows(execute("SELECT k, asciival from mv_timestampval WHERE timestampval = fromJson(?)", "\"2014-01-01\""), row(0, "ascii text")); + + // ================ timeuuid ================ + updateMV("INSERT INTO %s (k, timeuuidval) VALUES (?, fromJson(?))", 0, "\"6bddc89a-5644-11e4-97fc-56847afe9799\""); + assertRows(execute("SELECT k, timeuuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))); + + updateMV("INSERT INTO %s (k, timeuuidval) VALUES (?, fromJson(?))", 0, "\"6BDDC89A-5644-11E4-97FC-56847AFE9799\""); + assertRows(execute("SELECT k, timeuuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))); + assertRows(execute("SELECT k, asciival from mv_timeuuidval WHERE timeuuidval = fromJson(?)", "\"6BDDC89A-5644-11E4-97FC-56847AFE9799\""), row(0, "ascii text")); + + // ================ uuidval ================ + updateMV("INSERT INTO %s (k, uuidval) VALUES (?, fromJson(?))", 0, "\"6bddc89a-5644-11e4-97fc-56847afe9799\""); + assertRows(execute("SELECT k, uuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))); + + updateMV("INSERT INTO %s (k, uuidval) VALUES (?, fromJson(?))", 0, "\"6BDDC89A-5644-11E4-97FC-56847AFE9799\""); + assertRows(execute("SELECT k, uuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))); + assertRows(execute("SELECT k, asciival from mv_uuidval WHERE uuidval = fromJson(?)", "\"6BDDC89A-5644-11E4-97FC-56847AFE9799\""), row(0, "ascii text")); + + // ================ varint ================ + updateMV("INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "123123123123"); + assertRows(execute("SELECT k, varintval FROM %s WHERE k = ?", 0), row(0, new BigInteger("123123123123"))); + assertRows(execute("SELECT k, asciival from mv_varintval WHERE varintval = fromJson(?)", "123123123123"), row(0, "ascii text")); + + // accept strings for numbers that cannot be represented as longs + updateMV("INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "\"1234567890123456789012345678901234567890\""); + assertRows(execute("SELECT k, varintval FROM %s WHERE k = ?", 0), row(0, new BigInteger("1234567890123456789012345678901234567890"))); + assertRows(execute("SELECT k, asciival from mv_varintval WHERE varintval = fromJson(?)", "\"1234567890123456789012345678901234567890\""), row(0, "ascii text")); + + // ================ lists ================ + updateMV("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[1, 2, 3]"); + assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(1, 2, 3))); + assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(1, 2, 3))); + + updateMV("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[1]"); + assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(1))); + assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(1))); + + updateMV("UPDATE %s SET listval = listval + fromJson(?) WHERE k = ?", "[2]", 0); + assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(1, 2))); + assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(1, 2))); + + updateMV("UPDATE %s SET listval = fromJson(?) + listval WHERE k = ?", "[0]", 0); + assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(0, 1, 2))); + assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(0, 1, 2))); + + updateMV("UPDATE %s SET listval[1] = fromJson(?) WHERE k = ?", "10", 0); + assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(0, 10, 2))); + assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(0, 10, 2))); + + updateMV("DELETE listval[1] FROM %s WHERE k = ?", 0); + assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(0, 2))); + assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(0, 2))); + + updateMV("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[]"); + assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, null)); + assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, null)); + + // frozen + updateMV("INSERT INTO %s (k, frozenlistval) VALUES (?, fromJson(?))", 0, "[1, 2, 3]"); + assertRows(execute("SELECT k, frozenlistval FROM %s WHERE k = ?", 0), row(0, list(1, 2, 3))); + assertRows(execute("SELECT k, frozenlistval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(1, 2, 3))); + assertRows(execute("SELECT k, textval from mv_frozenlistval where frozenlistval = fromJson(?)", "[1, 2, 3]"), row(0, "abcd")); + + updateMV("INSERT INTO %s (k, frozenlistval) VALUES (?, fromJson(?))", 0, "[3, 2, 1]"); + assertRows(execute("SELECT k, frozenlistval FROM %s WHERE k = ?", 0), row(0, list(3, 2, 1))); + assertRows(execute("SELECT k, textval from mv_frozenlistval where frozenlistval = fromJson(?)", "[1, 2, 3]")); + assertRows(execute("SELECT k, textval from mv_frozenlistval where frozenlistval = fromJson(?)", "[3, 2, 1]"), row(0, "abcd")); + assertRows(execute("SELECT k, frozenlistval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(3, 2, 1))); + + updateMV("INSERT INTO %s (k, frozenlistval) VALUES (?, fromJson(?))", 0, "[]"); + assertRows(execute("SELECT k, frozenlistval FROM %s WHERE k = ?", 0), row(0, list())); + assertRows(execute("SELECT k, frozenlistval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list())); + + // ================ sets ================ + updateMV("INSERT INTO %s (k, setval) VALUES (?, fromJson(?))", + 0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]"); + assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0), + row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))) + ); + assertRows(execute("SELECT k, setval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), + row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))); + + // duplicates are okay, just like in CQL + updateMV("INSERT INTO %s (k, setval) VALUES (?, fromJson(?))", + 0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]"); + assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0), + row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))) + ); + assertRows(execute("SELECT k, setval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), + row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))); + + updateMV("UPDATE %s SET setval = setval + fromJson(?) WHERE k = ?", "[\"6bddc89a-5644-0000-97fc-56847afe9799\"]", 0); + assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0), + row(0, set(UUID.fromString("6bddc89a-5644-0000-97fc-56847afe9799"), UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))) + ); + assertRows(execute("SELECT k, setval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), + row(0, set(UUID.fromString("6bddc89a-5644-0000-97fc-56847afe9799"), UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))); + + updateMV("UPDATE %s SET setval = setval - fromJson(?) WHERE k = ?", "[\"6bddc89a-5644-0000-97fc-56847afe9799\"]", 0); + assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0), + row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))) + ); + assertRows(execute("SELECT k, setval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), + row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))); + + updateMV("INSERT INTO %s (k, setval) VALUES (?, fromJson(?))", 0, "[]"); + assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0), row(0, null)); + assertRows(execute("SELECT k, setval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), + row(0, null)); + + + // frozen + updateMV("INSERT INTO %s (k, frozensetval) VALUES (?, fromJson(?))", + 0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]"); + assertRows(execute("SELECT k, frozensetval FROM %s WHERE k = ?", 0), + row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))) + ); + assertRows(execute("SELECT k, frozensetval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), + row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))); + + updateMV("INSERT INTO %s (k, frozensetval) VALUES (?, fromJson(?))", + 0, "[\"6bddc89a-0000-11e4-97fc-56847afe9799\", \"6bddc89a-5644-11e4-97fc-56847afe9798\"]"); + assertRows(execute("SELECT k, frozensetval FROM %s WHERE k = ?", 0), + row(0, set(UUID.fromString("6bddc89a-0000-11e4-97fc-56847afe9799"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798")))) + ); + assertRows(execute("SELECT k, frozensetval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), + row(0, set(UUID.fromString("6bddc89a-0000-11e4-97fc-56847afe9799"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"))))); + + // ================ maps ================ + updateMV("INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": 2}"); + assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0), row(0, map("a", 1, "b", 2))); + assertRows(execute("SELECT k, mapval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, map("a", 1, "b", 2))); + + updateMV("UPDATE %s SET mapval[?] = ? WHERE k = ?", "c", 3, 0); + assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0), + row(0, map("a", 1, "b", 2, "c", 3)) + ); + assertRows(execute("SELECT k, mapval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), + row(0, map("a", 1, "b", 2, "c", 3))); + + updateMV("UPDATE %s SET mapval[?] = ? WHERE k = ?", "b", 10, 0); + assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0), + row(0, map("a", 1, "b", 10, "c", 3)) + ); + assertRows(execute("SELECT k, mapval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), + row(0, map("a", 1, "b", 10, "c", 3))); + + updateMV("DELETE mapval[?] FROM %s WHERE k = ?", "b", 0); + assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0), + row(0, map("a", 1, "c", 3)) + ); + assertRows(execute("SELECT k, mapval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), + row(0, map("a", 1, "c", 3))); + + updateMV("INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "{}"); + assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0), row(0, null)); + assertRows(execute("SELECT k, mapval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), + row(0, null)); + + // frozen + updateMV("INSERT INTO %s (k, frozenmapval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": 2}"); + assertRows(execute("SELECT k, frozenmapval FROM %s WHERE k = ?", 0), row(0, map("a", 1, "b", 2))); + assertRows(execute("SELECT k, textval FROM mv_frozenmapval WHERE frozenmapval = fromJson(?)", "{\"a\": 1, \"b\": 2}"), row(0, "abcd")); + + updateMV("INSERT INTO %s (k, frozenmapval) VALUES (?, fromJson(?))", 0, "{\"b\": 2, \"a\": 3}"); + assertRows(execute("SELECT k, frozenmapval FROM %s WHERE k = ?", 0), row(0, map("a", 3, "b", 2))); + assertRows(execute("SELECT k, frozenmapval FROM %s WHERE k = ?", 0), row(0, map("a", 3, "b", 2))); + + // ================ tuples ================ + updateMV("INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))", 0, "[1, \"foobar\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]"); + assertRows(execute("SELECT k, tupleval FROM %s WHERE k = ?", 0), + row(0, tuple(1, "foobar", UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))) + ); + assertRows(execute("SELECT k, textval FROM mv_tupleval WHERE tupleval = ?", tuple(1, "foobar", UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))), + row(0, "abcd")); + + updateMV("INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))", 0, "[1, null, \"6bddc89a-5644-11e4-97fc-56847afe9799\"]"); + assertRows(execute("SELECT k, tupleval FROM %s WHERE k = ?", 0), + row(0, tuple(1, null, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))) + ); + assertRows(execute("SELECT k, textval FROM mv_tupleval WHERE tupleval = ?", tuple(1, "foobar", UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))); + assertRows(execute("SELECT k, textval FROM mv_tupleval WHERE tupleval = ?", tuple(1, null, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))), + row(0, "abcd")); + + // ================ UDTs ================ + updateMV("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}"); + assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0), + row(0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("bar", "foo")) + ); + assertRows(execute("SELECT k, textval FROM mv_udtval WHERE udtval = fromJson(?)", "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}"), + row(0, "abcd")); + + // order of fields shouldn't matter + updateMV("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"a\": 1, \"c\": [\"foo\", \"bar\"]}"); + assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0), + row(0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("bar", "foo")) + ); + assertRows(execute("SELECT k, textval FROM mv_udtval WHERE udtval = fromJson(?)", "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}"), + row(0, "abcd")); + + // test nulls + updateMV("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": null, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}"); + assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0), + row(0, null, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("bar", "foo")) + ); + assertRows(execute("SELECT k, textval FROM mv_udtval WHERE udtval = fromJson(?)", "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}")); + assertRows(execute("SELECT k, textval FROM mv_udtval WHERE udtval = fromJson(?)", "{\"a\": null, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}"), + row(0, "abcd")); + + // test missing fields + updateMV("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\"}"); + assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0), + row(0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), null) + ); + assertRows(execute("SELECT k, textval FROM mv_udtval WHERE udtval = fromJson(?)", "{\"a\": null, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}")); + assertRows(execute("SELECT k, textval FROM mv_udtval WHERE udtval = fromJson(?)", "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\"}"), + row(0, "abcd")); + } + + + @Test + public void ttlTest() throws Throwable + { + createTable("CREATE TABLE %s (" + + "a int," + + "b int," + + "c int," + + "d int," + + "PRIMARY KEY (a, b))"); + + executeNet(protocolVersion, "USE " + keyspace()); + + createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE c IS NOT NULL AND a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (c, a, b)"); + + updateMV("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TTL 5", 1, 1, 1, 1); + + Thread.sleep(TimeUnit.SECONDS.toMillis(3)); + updateMV("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, 1, 2); + + Thread.sleep(TimeUnit.SECONDS.toMillis(3)); + List<Row> results = executeNet(protocolVersion, "SELECT d FROM mv WHERE c = 2 AND a = 1 AND b = 1").all(); + Assert.assertEquals(1, results.size()); + Assert.assertTrue("There should be a null result given back due to ttl expiry", results.get(0).isNull(0)); + } +}
