merge from 1.1
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/225cb0e7 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/225cb0e7 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/225cb0e7 Branch: refs/heads/cassandra-1.2 Commit: 225cb0e7fac97c302783d2334b5b57926bbfa346 Parents: c7eb146 feae9ef Author: Jonathan Ellis <jbel...@apache.org> Authored: Tue Apr 9 21:25:13 2013 -0500 Committer: Jonathan Ellis <jbel...@apache.org> Committed: Tue Apr 9 21:25:13 2013 -0500 ---------------------------------------------------------------------- CHANGES.txt | 2 ++ .../org/apache/cassandra/service/CacheService.java | 3 ++- 2 files changed, 4 insertions(+), 1 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/225cb0e7/CHANGES.txt ---------------------------------------------------------------------- diff --cc CHANGES.txt index 32aba15,59a7443..66279f0 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,76 -1,5 +1,78 @@@ -1.1.11 +1.2.5 + * Include fatal errors in trace events (CASSANDRA-5447) + * Ensure that PerRowSecondaryIndex is notified of row-level deletes + (CASSANDRA-5445) ++Merged from 1.1: + * Fix trying to load deleted row into row cache on startup (CASSANDRA-4463) + + +1.2.4 + * Ensure that PerRowSecondaryIndex updates see the most recent values + (CASSANDRA-5397) + * avoid duplicate index entries ind PrecompactedRow and + ParallelCompactionIterable (CASSANDRA-5395) + * remove the index entry on oldColumn when new column is a tombstone + (CASSANDRA-5395) + * Change default stream throughput from 400 to 200 mbps (CASSANDRA-5036) + * Gossiper logs DOWN for symmetry with UP (CASSANDRA-5187) + * Fix mixing prepared statements between keyspaces (CASSANDRA-5352) + * Fix consistency level during bootstrap - strike 3 (CASSANDRA-5354) + * Fix transposed arguments in AlreadyExistsException (CASSANDRA-5362) + * Improve asynchronous hint delivery (CASSANDRA-5179) + * Fix Guava dependency version (12.0 -> 13.0.1) for Maven (CASSANDRA-5364) + * Validate that provided CQL3 collection value are < 64K (CASSANDRA-5355) + * Make upgradeSSTable skip current version sstables by default (CASSANDRA-5366) + * Optimize min/max timestamp collection (CASSANDRA-5373) + * Invalid streamId in cql binary protocol when using invalid CL + (CASSANDRA-5164) + * Fix validation for IN where clauses with collections (CASSANDRA-5376) + * Copy resultSet on count query to avoid ConcurrentModificationException + (CASSANDRA-5382) + * Correctly typecheck in CQL3 even with ReversedType (CASSANDRA-5386) + * Fix streaming compressed files when using encryption (CASSANDRA-5391) + * cassandra-all 1.2.0 pom missing netty dependency (CASSANDRA-5392) + * Fix writetime/ttl functions on null values (CASSANDRA-5341) + * Fix NPE during cql3 select with token() (CASSANDRA-5404) + * IndexHelper.skipBloomFilters won't skip non-SHA filters (CASSANDRA-5385) + * cqlsh: Print maps ordered by key, sort sets (CASSANDRA-5413) + * Add null syntax support in CQL3 for inserts (CASSANDRA-3783) + * Allow unauthenticated set_keyspace() calls (CASSANDRA-5423) + * Fix potential incremental backups race (CASSANDRA-5410) + * Fix prepared BATCH statements with batch-level timestamps (CASSANDRA-5415) + * Allow overriding superuser setup delay (CASSANDRA-5430) + * cassandra-shuffle with JMX usernames and passwords (CASSANDRA-5431) +Merged from 1.1: + * cli: Quote ks and cf names in schema output when needed (CASSANDRA-5052) + * Fix bad default for min/max timestamp in SSTableMetadata (CASSANDRA-5372) + * Fix cf name extraction from manifest in Directories.migrateFile() + (CASSANDRA-5242) + * Support pluggable internode authentication (CASSANDRA-5401) + + +1.2.3 + * add check for sstable overlap within a level on startup (CASSANDRA-5327) + * replace ipv6 colons in jmx object names (CASSANDRA-5298, 5328) + * Avoid allocating SSTableBoundedScanner during repair when the range does + not intersect the sstable (CASSANDRA-5249) + * Don't lowercase property map keys (this breaks NTS) (CASSANDRA-5292) + * Fix composite comparator with super columns (CASSANDRA-5287) + * Fix insufficient validation of UPDATE queries against counter cfs + (CASSANDRA-5300) + * Fix PropertyFileSnitch default DC/Rack behavior (CASSANDRA-5285) + * Handle null values when executing prepared statement (CASSANDRA-5081) + * Add netty to pom dependencies (CASSANDRA-5181) + * Include type arguments in Thrift CQLPreparedResult (CASSANDRA-5311) + * Fix compaction not removing columns when bf_fp_ratio is 1 (CASSANDRA-5182) + * cli: Warn about missing CQL3 tables in schema descriptions (CASSANDRA-5309) + * Re-enable unknown option in replication/compaction strategies option for + backward compatibility (CASSANDRA-4795) + * Add binary protocol support to stress (CASSANDRA-4993) + * cqlsh: Fix COPY FROM value quoting and null handling (CASSANDRA-5305) + * Fix repair -pr for vnodes (CASSANDRA-5329) + * Relax CL for auth queries for non-default users (CASSANDRA-5310) + * Fix AssertionError during repair (CASSANDRA-5245) + * Don't announce migrations to pre-1.2 nodes (CASSANDRA-5334) +Merged from 1.1: * Update offline scrub for 1.0 -> 1.1 directory structure (CASSANDRA-5195) * add tmp flag to Descriptor hashcode (CASSANDRA-4021) * fix logging of "Found table data in data directories" when only system tables http://git-wip-us.apache.org/repos/asf/cassandra/blob/225cb0e7/src/java/org/apache/cassandra/service/CacheService.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/service/CacheService.java index 21f007c,d63e775..bc218c4 --- a/src/java/org/apache/cassandra/service/CacheService.java +++ b/src/java/org/apache/cassandra/service/CacheService.java @@@ -324,96 -290,4 +324,97 @@@ public class CacheService implements Ca FBUtilities.waitOnFutures(futures); logger.debug("cache saves completed"); } + + public class RowCacheSerializer implements CacheSerializer<RowCacheKey, IRowCacheEntry> + { + public void serialize(RowCacheKey key, DataOutput out) throws IOException + { + ByteBufferUtil.writeWithLength(key.key, out); + } + + public Future<Pair<RowCacheKey, IRowCacheEntry>> deserialize(DataInputStream in, final ColumnFamilyStore cfs) throws IOException + { + final ByteBuffer buffer = ByteBufferUtil.readWithLength(in); + return StageManager.getStage(Stage.READ).submit(new Callable<Pair<RowCacheKey, IRowCacheEntry>>() + { + public Pair<RowCacheKey, IRowCacheEntry> call() throws Exception + { + DecoratedKey key = cfs.partitioner.decorateKey(buffer); + ColumnFamily data = cfs.getTopLevelColumns(QueryFilter.getIdentityFilter(key, new QueryPath(cfs.columnFamily)), Integer.MIN_VALUE, true); + return Pair.create(new RowCacheKey(cfs.metadata.cfId, key), (IRowCacheEntry) data); + } + }); + } + + public void load(Set<ByteBuffer> buffers, ColumnFamilyStore cfs) + { + for (ByteBuffer key : buffers) + { + DecoratedKey dk = cfs.partitioner.decorateKey(key); + ColumnFamily data = cfs.getTopLevelColumns(QueryFilter.getIdentityFilter(dk, new QueryPath(cfs.columnFamily)), Integer.MIN_VALUE, true); - rowCache.put(new RowCacheKey(cfs.metadata.cfId, dk), data); ++ if (data != null) ++ rowCache.put(new RowCacheKey(cfs.metadata.cfId, dk), data); + } + } + } + + public class KeyCacheSerializer implements CacheSerializer<KeyCacheKey, RowIndexEntry> + { + public void serialize(KeyCacheKey key, DataOutput out) throws IOException + { + RowIndexEntry entry = CacheService.instance.keyCache.get(key); + if (entry == null) + return; + ByteBufferUtil.writeWithLength(key.key, out); + Descriptor desc = key.desc; + out.writeInt(desc.generation); + out.writeBoolean(desc.version.hasPromotedIndexes); + if (!desc.version.hasPromotedIndexes) + return; + RowIndexEntry.serializer.serialize(entry, out); + } + + public Future<Pair<KeyCacheKey, RowIndexEntry>> deserialize(DataInputStream input, ColumnFamilyStore cfs) throws IOException + { + ByteBuffer key = ByteBufferUtil.readWithLength(input); + int generation = input.readInt(); + SSTableReader reader = findDesc(generation, cfs.getSSTables()); + if (reader == null) + { + RowIndexEntry.serializer.skipPromotedIndex(input); + return null; + } + RowIndexEntry entry; + if (input.readBoolean()) + entry = RowIndexEntry.serializer.deserialize(input, reader.descriptor.version); + else + entry = reader.getPosition(reader.partitioner.decorateKey(key), Operator.EQ); + return Futures.immediateFuture(Pair.create(new KeyCacheKey(reader.descriptor, key), entry)); + } + + private SSTableReader findDesc(int generation, Collection<SSTableReader> collection) + { + for (SSTableReader sstable : collection) + { + if (sstable.descriptor.generation == generation) + return sstable; + } + return null; + } + + public void load(Set<ByteBuffer> buffers, ColumnFamilyStore cfs) + { + for (ByteBuffer key : buffers) + { + DecoratedKey dk = cfs.partitioner.decorateKey(key); + + for (SSTableReader sstable : cfs.getSSTables()) + { + RowIndexEntry entry = sstable.getPosition(dk, Operator.EQ); + if (entry != null) + keyCache.put(new KeyCacheKey(sstable.descriptor, key), entry); + } + } + } + } }