Repository: cassandra Updated Branches: refs/heads/cassandra-3.0 f27cc7ef3 -> 4beb54da5
http://git-wip-us.apache.org/repos/asf/cassandra/blob/4beb54da/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 02c0871..ee86f9d 100644 --- a/src/java/org/apache/cassandra/thrift/CassandraServer.java +++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java @@ -85,7 +85,7 @@ public class CassandraServer implements Cassandra.Iface return ThriftSessionManager.instance.currentSession(); } - protected PartitionIterator read(List<SinglePartitionReadCommand<?>> commands, org.apache.cassandra.db.ConsistencyLevel consistency_level, ClientState cState) + protected PartitionIterator read(List<SinglePartitionReadCommand> commands, org.apache.cassandra.db.ConsistencyLevel consistency_level, ClientState cState) throws org.apache.cassandra.exceptions.InvalidRequestException, UnavailableException, TimedOutException { try @@ -257,7 +257,7 @@ public class CassandraServer implements Cassandra.Iface : result; } - private Map<ByteBuffer, List<ColumnOrSuperColumn>> getSlice(List<SinglePartitionReadCommand<?>> commands, boolean subColumnsOnly, int cellLimit, org.apache.cassandra.db.ConsistencyLevel consistency_level, ClientState cState) + private Map<ByteBuffer, List<ColumnOrSuperColumn>> getSlice(List<SinglePartitionReadCommand> commands, boolean subColumnsOnly, int cellLimit, org.apache.cassandra.db.ConsistencyLevel consistency_level, ClientState cState) throws org.apache.cassandra.exceptions.InvalidRequestException, UnavailableException, TimedOutException { try (PartitionIterator results = read(commands, consistency_level, cState)) @@ -551,7 +551,7 @@ public class CassandraServer implements Cassandra.Iface org.apache.cassandra.db.ConsistencyLevel consistencyLevel = ThriftConversion.fromThrift(consistency_level); consistencyLevel.validateForRead(keyspace); - List<SinglePartitionReadCommand<?>> commands = new ArrayList<>(keys.size()); + List<SinglePartitionReadCommand> commands = new ArrayList<>(keys.size()); ColumnFilter columnFilter = makeColumnFilter(metadata, column_parent, predicate); ClusteringIndexFilter filter = toInternalFilter(metadata, column_parent, predicate); DataLimits limits = getLimits(1, metadata.isSuper() && !column_parent.isSetSuper_column(), predicate); @@ -641,7 +641,7 @@ public class CassandraServer implements Cassandra.Iface } DecoratedKey dk = metadata.decorateKey(key); - SinglePartitionReadCommand<?> command = SinglePartitionReadCommand.create(true, metadata, FBUtilities.nowInSeconds(), columns, RowFilter.NONE, DataLimits.NONE, dk, filter); + SinglePartitionReadCommand command = SinglePartitionReadCommand.create(true, metadata, FBUtilities.nowInSeconds(), columns, RowFilter.NONE, DataLimits.NONE, dk, filter); try (RowIterator result = PartitionIterators.getOnlyElement(read(Arrays.asList(command), consistencyLevel, cState), command)) { @@ -2437,8 +2437,8 @@ public class CassandraServer implements Cassandra.Iface ThriftValidation.validateKey(metadata, request.key); DecoratedKey dk = metadata.decorateKey(request.key); - SinglePartitionReadCommand<?> cmd = SinglePartitionReadCommand.create(true, metadata, FBUtilities.nowInSeconds(), columns, RowFilter.NONE, limits, dk, filter); - return getSlice(Collections.<SinglePartitionReadCommand<?>>singletonList(cmd), + SinglePartitionReadCommand cmd = SinglePartitionReadCommand.create(true, metadata, FBUtilities.nowInSeconds(), columns, RowFilter.NONE, limits, dk, filter); + return getSlice(Collections.<SinglePartitionReadCommand>singletonList(cmd), false, limits.perPartitionCount(), consistencyLevel, @@ -2525,7 +2525,7 @@ public class CassandraServer implements Cassandra.Iface // We want to know if the partition exists, so just fetch a single cell. ClusteringIndexSliceFilter filter = new ClusteringIndexSliceFilter(Slices.ALL, false); DataLimits limits = DataLimits.thriftLimits(1, 1); - return new SinglePartitionSliceCommand(false, 0, true, metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, limits, key, filter); + return new SinglePartitionReadCommand(false, 0, true, metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, limits, key, filter); } // Gather the clustering for the expected values and query those. http://git-wip-us.apache.org/repos/asf/cassandra/blob/4beb54da/test/unit/org/apache/cassandra/db/KeyspaceTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/db/KeyspaceTest.java b/test/unit/org/apache/cassandra/db/KeyspaceTest.java index 2e79dfe..d864fa3 100644 --- a/test/unit/org/apache/cassandra/db/KeyspaceTest.java +++ b/test/unit/org/apache/cassandra/db/KeyspaceTest.java @@ -133,7 +133,7 @@ public class KeyspaceTest extends CQLTester Clustering endClustering = new Clustering(ByteBufferUtil.bytes(sliceEnd)); Slices slices = Slices.with(cfs.getComparator(), Slice.make(startClustering, endClustering)); ClusteringIndexSliceFilter filter = new ClusteringIndexSliceFilter(slices, reversed); - SinglePartitionSliceCommand command = singlePartitionSlice(cfs, key, filter, limit); + SinglePartitionReadCommand command = singlePartitionSlice(cfs, key, filter, limit); try (ReadOrderGroup orderGroup = command.startOrderGroup(); PartitionIterator iterator = command.executeInternal(orderGroup)) { @@ -207,7 +207,7 @@ public class KeyspaceTest extends CQLTester PartitionColumns columns = PartitionColumns.of(cfs.metadata.getColumnDefinition(new ColumnIdentifier("c", false))); ClusteringIndexSliceFilter filter = new ClusteringIndexSliceFilter(Slices.ALL, false); - SinglePartitionSliceCommand command = singlePartitionSlice(cfs, "0", filter, null); + SinglePartitionReadCommand command = singlePartitionSlice(cfs, "0", filter, null); try (ReadOrderGroup orderGroup = command.startOrderGroup(); PartitionIterator iterator = command.executeInternal(orderGroup)) { try (RowIterator rowIterator = iterator.next()) @@ -258,12 +258,12 @@ public class KeyspaceTest extends CQLTester return new ClusteringIndexSliceFilter(slices, reversed); } - private static SinglePartitionSliceCommand singlePartitionSlice(ColumnFamilyStore cfs, String key, ClusteringIndexSliceFilter filter, Integer rowLimit) + private static SinglePartitionReadCommand singlePartitionSlice(ColumnFamilyStore cfs, String key, ClusteringIndexSliceFilter filter, Integer rowLimit) { DataLimits limit = rowLimit == null ? DataLimits.NONE : DataLimits.cqlLimits(rowLimit); - return new SinglePartitionSliceCommand( + return SinglePartitionReadCommand.create( cfs.metadata, FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata), RowFilter.NONE, limit, Util.dk(key), filter); } @@ -287,7 +287,7 @@ public class KeyspaceTest extends CQLTester { ClusteringIndexSliceFilter filter = slices(cfs, 5, null, false); - SinglePartitionSliceCommand command = singlePartitionSlice(cfs, "0", filter, 2); + SinglePartitionReadCommand command = singlePartitionSlice(cfs, "0", filter, 2); assertRowsInResult(cfs, command, 5, 7); command = singlePartitionSlice(cfs, "0", slices(cfs, 4, null, false), 2); @@ -329,7 +329,7 @@ public class KeyspaceTest extends CQLTester for (int round = 0; round < 2; round++) { - SinglePartitionSliceCommand command = singlePartitionSlice(cfs, "0", slices(cfs, null, null, false), 2); + SinglePartitionReadCommand command = singlePartitionSlice(cfs, "0", slices(cfs, null, null, false), 2); assertRowsInResult(cfs, command, 0, 1); command = singlePartitionSlice(cfs, "0", slices(cfs, 1, null, false), 1); @@ -358,7 +358,7 @@ public class KeyspaceTest extends CQLTester for (int round = 0; round < 2; round++) { - SinglePartitionSliceCommand command = singlePartitionSlice(cfs, "0", slices(cfs, 2, null, false), 3); + SinglePartitionReadCommand command = singlePartitionSlice(cfs, "0", slices(cfs, 2, null, false), 3); assertRowsInResult(cfs, command, -1, -1, 4); if (round == 0) @@ -409,7 +409,7 @@ public class KeyspaceTest extends CQLTester ((ClearableHistogram)cfs.metric.sstablesPerReadHistogram.cf).clear(); - SinglePartitionSliceCommand command = singlePartitionSlice(cfs, "0", slices(cfs, null, 1499, false), 1000); + SinglePartitionReadCommand command = singlePartitionSlice(cfs, "0", slices(cfs, null, 1499, false), 1000); int[] expectedValues = new int[500]; for (int i = 0; i < 500; i++) expectedValues[i] = i + 1000; @@ -470,17 +470,17 @@ public class KeyspaceTest extends CQLTester private void validateSliceLarge(ColumnFamilyStore cfs) { ClusteringIndexSliceFilter filter = slices(cfs, 1000, null, false); - SinglePartitionSliceCommand command = new SinglePartitionSliceCommand( + SinglePartitionReadCommand command = SinglePartitionReadCommand.create( cfs.metadata, FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata), RowFilter.NONE, DataLimits.cqlLimits(3), Util.dk("0"), filter); assertRowsInResult(cfs, command, 1000, 1001, 1002); filter = slices(cfs, 1195, null, false); - command = new SinglePartitionSliceCommand( + command = SinglePartitionReadCommand.create( cfs.metadata, FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata), RowFilter.NONE, DataLimits.cqlLimits(3), Util.dk("0"), filter); assertRowsInResult(cfs, command, 1195, 1196, 1197); filter = slices(cfs, null, 1996, true); - command = new SinglePartitionSliceCommand( + command = SinglePartitionReadCommand.create( cfs.metadata, FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata), RowFilter.NONE, DataLimits.cqlLimits(1000), Util.dk("0"), filter); int[] expectedValues = new int[997]; for (int i = 0, v = 1996; v >= 1000; i++, v--) @@ -488,22 +488,22 @@ public class KeyspaceTest extends CQLTester assertRowsInResult(cfs, command, expectedValues); filter = slices(cfs, 1990, null, false); - command = new SinglePartitionSliceCommand( + command = SinglePartitionReadCommand.create( cfs.metadata, FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata), RowFilter.NONE, DataLimits.cqlLimits(3), Util.dk("0"), filter); assertRowsInResult(cfs, command, 1990, 1991, 1992); filter = slices(cfs, null, null, true); - command = new SinglePartitionSliceCommand( + command = SinglePartitionReadCommand.create( cfs.metadata, FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata), RowFilter.NONE, DataLimits.cqlLimits(3), Util.dk("0"), filter); assertRowsInResult(cfs, command, 1999, 1998, 1997); filter = slices(cfs, null, 9000, true); - command = new SinglePartitionSliceCommand( + command = SinglePartitionReadCommand.create( cfs.metadata, FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata), RowFilter.NONE, DataLimits.cqlLimits(3), Util.dk("0"), filter); assertRowsInResult(cfs, command, 1999, 1998, 1997); filter = slices(cfs, 9000, null, false); - command = new SinglePartitionSliceCommand( + command = SinglePartitionReadCommand.create( cfs.metadata, FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata), RowFilter.NONE, DataLimits.cqlLimits(3), Util.dk("0"), filter); assertRowsInResult(cfs, command); } http://git-wip-us.apache.org/repos/asf/cassandra/blob/4beb54da/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java index a6ee48f..d0cc890 100644 --- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java +++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java @@ -210,7 +210,7 @@ public class RangeTombstoneTest sb.add(Slice.Bound.create(cfs.getComparator(), true, true, 1), Slice.Bound.create(cfs.getComparator(), false, true, 10)); sb.add(Slice.Bound.create(cfs.getComparator(), true, true, 16), Slice.Bound.create(cfs.getComparator(), false, true, 20)); - partition = Util.getOnlyPartitionUnfiltered(SinglePartitionSliceCommand.create(cfs.metadata, FBUtilities.nowInSeconds(), Util.dk(key), sb.build())); + partition = Util.getOnlyPartitionUnfiltered(SinglePartitionReadCommand.create(cfs.metadata, FBUtilities.nowInSeconds(), Util.dk(key), sb.build())); rt = rangeTombstones(partition); assertEquals(2, rt.size()); } http://git-wip-us.apache.org/repos/asf/cassandra/blob/4beb54da/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java b/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java index 46c7810..7cacb5e 100644 --- a/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java +++ b/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java @@ -96,7 +96,7 @@ public class SinglePartitionSliceCommandTest ByteBuffer zero = ByteBufferUtil.bytes(0); Slices slices = Slices.with(cfm.comparator, Slice.make(Slice.Bound.inclusiveStartOf(zero), Slice.Bound.inclusiveEndOf(zero))); ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(slices, false); - ReadCommand cmd = new SinglePartitionSliceCommand(false, MessagingService.VERSION_30, true, cfm, + ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm, FBUtilities.nowInSeconds(), columnFilter, RowFilter.NONE, @@ -146,13 +146,13 @@ public class SinglePartitionSliceCommandTest ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(s)); ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.NONE, false); - ReadCommand cmd = new SinglePartitionSliceCommand(false, MessagingService.VERSION_30, true, cfm, - FBUtilities.nowInSeconds(), - columnFilter, - RowFilter.NONE, - DataLimits.NONE, - key, - sliceFilter); + ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm, + FBUtilities.nowInSeconds(), + columnFilter, + RowFilter.NONE, + DataLimits.NONE, + key, + sliceFilter); // check raw iterator for static cell try (ReadOrderGroup orderGroup = cmd.startOrderGroup(); UnfilteredPartitionIterator pi = cmd.executeLocally(orderGroup)) http://git-wip-us.apache.org/repos/asf/cassandra/blob/4beb54da/test/unit/org/apache/cassandra/service/QueryPagerTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/service/QueryPagerTest.java b/test/unit/org/apache/cassandra/service/QueryPagerTest.java index 0f79e84..15bc060 100644 --- a/test/unit/org/apache/cassandra/service/QueryPagerTest.java +++ b/test/unit/org/apache/cassandra/service/QueryPagerTest.java @@ -146,12 +146,12 @@ public class QueryPagerTest return builder.withPagingLimit(100).build(); } - private static SinglePartitionSliceCommand sliceQuery(String key, String start, String end, int count) + private static SinglePartitionReadCommand sliceQuery(String key, String start, String end, int count) { return sliceQuery(key, start, end, false, count); } - private static SinglePartitionSliceCommand sliceQuery(String key, String start, String end, boolean reversed, int count) + private static SinglePartitionReadCommand sliceQuery(String key, String start, String end, boolean reversed, int count) { ClusteringComparator cmp = cfs().getComparator(); CFMetaData metadata = cfs().metadata; @@ -159,9 +159,7 @@ public class QueryPagerTest Slice slice = Slice.make(cmp.make(start), cmp.make(end)); ClusteringIndexSliceFilter filter = new ClusteringIndexSliceFilter(Slices.with(cmp, slice), reversed); - SinglePartitionSliceCommand command = new SinglePartitionSliceCommand(cfs().metadata, FBUtilities.nowInSeconds(), ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, Util.dk(key), filter); - - return command; + return SinglePartitionReadCommand.create(cfs().metadata, FBUtilities.nowInSeconds(), ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, Util.dk(key), filter); } private static ReadCommand rangeNamesQuery(String keyStart, String keyEnd, int count, String... names) @@ -305,7 +303,7 @@ public class QueryPagerTest public void multiQueryTest(boolean testPagingState, int protocolVersion) throws Exception { - ReadQuery command = new SinglePartitionReadCommand.Group(new ArrayList<SinglePartitionReadCommand<?>>() + ReadQuery command = new SinglePartitionReadCommand.Group(new ArrayList<SinglePartitionReadCommand>() {{ add(sliceQuery("k1", "c2", "c6", 10)); add(sliceQuery("k4", "c3", "c5", 10)); @@ -427,7 +425,7 @@ public class QueryPagerTest for (int i = 0; i < 5; i++) executeInternal(String.format("INSERT INTO %s.%s (k, c, v) VALUES ('k%d', 'c%d', null)", keyspace, table, 0, i)); - ReadCommand command = SinglePartitionSliceCommand.create(cfs.metadata, FBUtilities.nowInSeconds(), Util.dk("k0"), Slice.ALL); + ReadCommand command = SinglePartitionReadCommand.create(cfs.metadata, FBUtilities.nowInSeconds(), Util.dk("k0"), Slice.ALL); QueryPager pager = command.getPager(null, Server.CURRENT_VERSION);