[2/2] git commit: Merge branch 'cassandra-1.1' into cassandra-1.2

2013-05-04 Thread marcuse
Merge branch 'cassandra-1.1' into cassandra-1.2

Conflicts:
CHANGES.txt
src/java/org/apache/cassandra/db/compaction/LeveledManifest.java


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

Branch: refs/heads/cassandra-1.2
Commit: ac1d7ee1c79897cbc867b37e712fe2d3b1ef1ea8
Parents: c007102 fe910e6
Author: Marcus Eriksson marc...@spotify.com
Authored: Sat May 4 20:31:50 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Sat May 4 20:31:50 2013 +0200

--
 CHANGES.txt|2 +-
 .../cassandra/db/compaction/LeveledManifest.java   |6 +-
 2 files changed, 6 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ac1d7ee1/CHANGES.txt
--
diff --cc CHANGES.txt
index 3c00e9b,5a559a8..4376a25
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,99 -1,11 +1,99 @@@
 -1.1.12
 +1.2.5
 + * fix compaction throttling bursty-ness (CASSANDRA-4316)
 + * reduce memory consumption of IndexSummary (CASSANDRA-5506)
 + * remove per-row column name bloom filters (CASSANDRA-5492)
 + * Include fatal errors in trace events (CASSANDRA-5447)
 + * Ensure that PerRowSecondaryIndex is notified of row-level deletes
 +   (CASSANDRA-5445)
 + * Allow empty blob literals in CQL3 (CASSANDRA-5452)
 + * Fix streaming RangeTombstones at column index boundary (CASSANDRA-5418)
 + * Fix preparing statements when current keyspace is not set (CASSANDRA-5468)
 + * Fix SemanticVersion.isSupportedBy minor/patch handling (CASSANDRA-5496)
 + * Don't provide oldCfId for post-1.1 system cfs (CASSANDRA-5490)
 + * Fix primary range ignores replication strategy (CASSANDRA-5424)
 + * Fix shutdown of binary protocol server (CASSANDRA-5507)
 + * Fix repair -snapshot not working (CASSANDRA-5512)
 + * Set isRunning flag later in binary protocol server (CASSANDRA-5467)
 + * Fix use of CQL3 functions with descending clustering order (CASSANDRA-5472)
 + * Prevent repair when protocol version does not match (CASSANDRA-5523)
 + * Disallow renaming columns one at a time for thrift table in CQL3
 +   (CASSANDRA-5531)
 + * cqlsh: add CLUSTERING ORDER BY support to DESCRIBE (CASSANDRA-5528)
 + * Add custom secondary index support to CQL3 (CASSANDRA-5484)
 +Merged from 1.1
   * Add retry mechanism to OTC for non-droppable_verbs (CASSANDRA-5393)
 - * Use allocator information to improve memtable memory usage estimate 
 + * Use allocator information to improve memtable memory usage estimate
 (CASSANDRA-5497)
 + * Fix trying to load deleted row into row cache on startup (CASSANDRA-4463)
- 
+  * fsync leveled manifest to avoid corruption (CASSANDRA-5535)
  
 -1.1.11
 - * 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 

[1/5] git commit: Update version for 1.2.5 release

2013-05-04 Thread marcuse
Updated Branches:
  refs/heads/trunk b505b90a8 - ec5f416a4


Update version for 1.2.5 release


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

Branch: refs/heads/trunk
Commit: de98541708b9a59e918029bb3ed50b0d986d24be
Parents: 64c0d1e
Author: Sylvain Lebresne sylv...@datastax.com
Authored: Fri May 3 17:54:04 2013 +0200
Committer: Sylvain Lebresne sylv...@datastax.com
Committed: Fri May 3 17:54:04 2013 +0200

--
 CHANGES.txt  |2 +-
 build.xml|2 +-
 debian/changelog |6 ++
 3 files changed, 8 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/de985417/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 21be695..3c00e9b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -14,7 +14,7 @@
  * Fix shutdown of binary protocol server (CASSANDRA-5507)
  * Fix repair -snapshot not working (CASSANDRA-5512)
  * Set isRunning flag later in binary protocol server (CASSANDRA-5467)
- * Fix use of CQL3 functions with descencind clustering order (CASSANDRA-5472)
+ * Fix use of CQL3 functions with descending clustering order (CASSANDRA-5472)
  * Prevent repair when protocol version does not match (CASSANDRA-5523)
  * Disallow renaming columns one at a time for thrift table in CQL3
(CASSANDRA-5531)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/de985417/build.xml
--
diff --git a/build.xml b/build.xml
index 05a6c84..cef16da 100644
--- a/build.xml
+++ b/build.xml
@@ -25,7 +25,7 @@
 property name=debuglevel value=source,lines,vars/
 
 !-- default version and SCM information --
-property name=base.version value=1.2.4/
+property name=base.version value=1.2.5/
 property name=scm.connection 
value=scm:git://git.apache.org/cassandra.git/
 property name=scm.developerConnection 
value=scm:git://git.apache.org/cassandra.git/
 property name=scm.url 
value=http://git-wip-us.apache.org/repos/asf?p=cassandra.git;a=tree/

http://git-wip-us.apache.org/repos/asf/cassandra/blob/de985417/debian/changelog
--
diff --git a/debian/changelog b/debian/changelog
index c8a62fa..81c12eb 100644
--- a/debian/changelog
+++ b/debian/changelog
@@ -1,3 +1,9 @@
+cassandra (1.2.5) unstable; urgency=low
+
+  * New release
+
+ -- Sylvain Lebresne slebre...@apache.org  Fri, 03 May 2013 17:52:01 +0200
+
 cassandra (1.2.4) unstable; urgency=low
 
   * New release



[2/5] git commit: fix repair test after CASSANDRA-5523

2013-05-04 Thread marcuse
fix repair test after CASSANDRA-5523


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

Branch: refs/heads/trunk
Commit: c007102c5b2fe2b6abd09a19cbc4b69f5e83ad0b
Parents: de98541
Author: Yuki Morishita yu...@apache.org
Authored: Fri May 3 12:53:16 2013 -0500
Committer: Yuki Morishita yu...@apache.org
Committed: Fri May 3 12:53:16 2013 -0500

--
 .../service/AntiEntropyServiceTestAbstract.java|3 +++
 1 files changed, 3 insertions(+), 0 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c007102c/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java
--
diff --git 
a/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java 
b/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java
index 7124ecb..fc0b832 100644
--- a/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java
+++ b/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java
@@ -43,6 +43,8 @@ import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.locator.TokenMetadata;
 import static org.apache.cassandra.service.AntiEntropyService.*;
+
+import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.MerkleTree;
@@ -100,6 +102,7 @@ public abstract class AntiEntropyServiceTestAbstract 
extends SchemaLoader
 
tmd.updateNormalToken(StorageService.getPartitioner().getMinimumToken(), 
REMOTE);
 assert tmd.isMember(REMOTE);
 
+MessagingService.instance().setVersion(REMOTE, 
MessagingService.current_version);
 Gossiper.instance.initializeNodeUnsafe(REMOTE, UUID.randomUUID(), 1);
 
 local_range = 
StorageService.instance.getPrimaryRangesForEndpoint(tablename, 
LOCAL).iterator().next();



[3/5] git commit: fsync leveled manifest to avoid it getting corrupt

2013-05-04 Thread marcuse
fsync leveled manifest to avoid it getting corrupt

Patch by marcuse, reviewed by jbellis and carlyeks for CASSANDRA-5535


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

Branch: refs/heads/trunk
Commit: fe910e6c90d81cc61c16859bcef9f0dcb42cc827
Parents: b4b4f6a
Author: Marcus Eriksson marc...@spotify.com
Authored: Sat May 4 18:49:04 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Sat May 4 20:28:03 2013 +0200

--
 CHANGES.txt|2 +-
 .../cassandra/db/compaction/LeveledManifest.java   |6 +-
 2 files changed, 6 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe910e6c/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 30a09c9..5a559a8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -2,7 +2,7 @@
  * Add retry mechanism to OTC for non-droppable_verbs (CASSANDRA-5393)
  * Use allocator information to improve memtable memory usage estimate 
(CASSANDRA-5497)
-
+ * fsync leveled manifest to avoid corruption (CASSANDRA-5535)
 
 1.1.11
  * Fix trying to load deleted row into row cache on startup (CASSANDRA-4463)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe910e6c/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
--
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java 
b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
index 5f22109..8ac7028 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
@@ -22,6 +22,7 @@ package org.apache.cassandra.db.compaction;
 
 
 import java.io.File;
+import java.io.FileOutputStream;
 import java.io.IOError;
 import java.io.IOException;
 import java.util.*;
@@ -534,7 +535,8 @@ public class LeveledManifest
 JsonFactory f = new JsonFactory();
 try
 {
-JsonGenerator g = f.createJsonGenerator(tmpFile, 
JsonEncoding.UTF8);
+FileOutputStream fos = new FileOutputStream(tmpFile);
+JsonGenerator g = f.createJsonGenerator(fos, JsonEncoding.UTF8);
 g.useDefaultPrettyPrinter();
 g.writeStartObject();
 g.writeArrayFieldStart(generations);
@@ -551,6 +553,8 @@ public class LeveledManifest
 }
 g.writeEndArray(); // for field generations
 g.writeEndObject(); // write global object
+g.flush();
+fos.getFD().sync();
 g.close();
 
 if (oldFile.exists()  manifestFile.exists())



[4/5] git commit: Merge branch 'cassandra-1.1' into cassandra-1.2

2013-05-04 Thread marcuse
Merge branch 'cassandra-1.1' into cassandra-1.2

Conflicts:
CHANGES.txt
src/java/org/apache/cassandra/db/compaction/LeveledManifest.java


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

Branch: refs/heads/trunk
Commit: ac1d7ee1c79897cbc867b37e712fe2d3b1ef1ea8
Parents: c007102 fe910e6
Author: Marcus Eriksson marc...@spotify.com
Authored: Sat May 4 20:31:50 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Sat May 4 20:31:50 2013 +0200

--
 CHANGES.txt|2 +-
 .../cassandra/db/compaction/LeveledManifest.java   |6 +-
 2 files changed, 6 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ac1d7ee1/CHANGES.txt
--
diff --cc CHANGES.txt
index 3c00e9b,5a559a8..4376a25
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,99 -1,11 +1,99 @@@
 -1.1.12
 +1.2.5
 + * fix compaction throttling bursty-ness (CASSANDRA-4316)
 + * reduce memory consumption of IndexSummary (CASSANDRA-5506)
 + * remove per-row column name bloom filters (CASSANDRA-5492)
 + * Include fatal errors in trace events (CASSANDRA-5447)
 + * Ensure that PerRowSecondaryIndex is notified of row-level deletes
 +   (CASSANDRA-5445)
 + * Allow empty blob literals in CQL3 (CASSANDRA-5452)
 + * Fix streaming RangeTombstones at column index boundary (CASSANDRA-5418)
 + * Fix preparing statements when current keyspace is not set (CASSANDRA-5468)
 + * Fix SemanticVersion.isSupportedBy minor/patch handling (CASSANDRA-5496)
 + * Don't provide oldCfId for post-1.1 system cfs (CASSANDRA-5490)
 + * Fix primary range ignores replication strategy (CASSANDRA-5424)
 + * Fix shutdown of binary protocol server (CASSANDRA-5507)
 + * Fix repair -snapshot not working (CASSANDRA-5512)
 + * Set isRunning flag later in binary protocol server (CASSANDRA-5467)
 + * Fix use of CQL3 functions with descending clustering order (CASSANDRA-5472)
 + * Prevent repair when protocol version does not match (CASSANDRA-5523)
 + * Disallow renaming columns one at a time for thrift table in CQL3
 +   (CASSANDRA-5531)
 + * cqlsh: add CLUSTERING ORDER BY support to DESCRIBE (CASSANDRA-5528)
 + * Add custom secondary index support to CQL3 (CASSANDRA-5484)
 +Merged from 1.1
   * Add retry mechanism to OTC for non-droppable_verbs (CASSANDRA-5393)
 - * Use allocator information to improve memtable memory usage estimate 
 + * Use allocator information to improve memtable memory usage estimate
 (CASSANDRA-5497)
 + * Fix trying to load deleted row into row cache on startup (CASSANDRA-4463)
- 
+  * fsync leveled manifest to avoid corruption (CASSANDRA-5535)
  
 -1.1.11
 - * 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 

[5/5] git commit: Merge branch 'cassandra-1.2' into trunk

2013-05-04 Thread marcuse
Merge branch 'cassandra-1.2' into trunk

Conflicts:
build.xml
src/java/org/apache/cassandra/db/compaction/LeveledManifest.java

test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java


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

Branch: refs/heads/trunk
Commit: ec5f416a466f3de8c1341cc426f25dcce7a7e723
Parents: b505b90 ac1d7ee
Author: Marcus Eriksson marc...@spotify.com
Authored: Sat May 4 20:34:34 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Sat May 4 20:42:25 2013 +0200

--
 CHANGES.txt|4 +-
 build.xml  |   89 ++-
 debian/changelog   |6 +
 .../service/AntiEntropyServiceTestAbstract.java|2 +
 4 files changed, 45 insertions(+), 56 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ec5f416a/CHANGES.txt
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ec5f416a/build.xml
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ec5f416a/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java
--
diff --cc 
test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java
index 04930d3,fc0b832..b120a3f
--- a/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java
+++ b/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java
@@@ -42,7 -42,9 +42,8 @@@ import org.apache.cassandra.dht.Token
  import org.apache.cassandra.gms.Gossiper;
  import org.apache.cassandra.locator.AbstractReplicationStrategy;
  import org.apache.cassandra.locator.TokenMetadata;
 -import static org.apache.cassandra.service.AntiEntropyService.*;
 -
 +import static org.apache.cassandra.service.ActiveRepairService.*;
+ import org.apache.cassandra.net.MessagingService;
  import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.FBUtilities;
  import org.apache.cassandra.utils.MerkleTree;



git commit: fix bad merge

2013-05-04 Thread marcuse
Updated Branches:
  refs/heads/trunk ec5f416a4 - 90764b0ac


fix bad merge


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

Branch: refs/heads/trunk
Commit: 90764b0ac7130a1fe50b7d30d0b2566ee8959f01
Parents: ec5f416
Author: Marcus Eriksson marc...@spotify.com
Authored: Sat May 4 21:23:24 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Sat May 4 21:23:24 2013 +0200

--
 build.xml |   89 ++--
 1 files changed, 54 insertions(+), 35 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/90764b0a/build.xml
--
diff --git a/build.xml b/build.xml
index 56a60cf..8650e2d 100644
--- a/build.xml
+++ b/build.xml
@@ -130,6 +130,10 @@
   exclude name=**/*-sources.jar/
 /fileset
 /path
+   
+   path id=cobertura.classpath
+   pathelement location=${cobertura.classes.dir}/
+   /path
 
   macrodef name=create-javadoc
 attribute name=destdir/
@@ -156,6 +160,7 @@
 mkdir dir=${test.lib}/
 mkdir dir=${test.classes}/
 mkdir dir=${build.src.gen-java}/
+mkdir dir=${build.dir.lib}/
 /target
 
 target name=clean description=Remove all locally created artifacts
@@ -251,24 +256,21 @@
can be run stand-alone from a binary release artifact.
 --
 target name=maven-ant-tasks-localrepo 
unless=maven-ant-tasks.jar.exists if=maven-ant-tasks.jar.local
-description=Fetch Maven ANT Tasks from Maven Local Repository
-  mkdir dir=${build.dir} /
+depends=init description=Fetch Maven ANT Tasks from Maven Local 
Repository
   copy 
file=${maven-ant-tasks.local}/${maven-ant-tasks.version}/maven-ant-tasks-${maven-ant-tasks.version}.jar

tofile=${build.dir}/maven-ant-tasks-${maven-ant-tasks.version}.jar/
   property name=maven-ant-tasks.jar.exists value=true/
 /target
 
-target name=maven-ant-tasks-download 
depends=maven-ant-tasks-localrepo unless=maven-ant-tasks.jar.exists
+target name=maven-ant-tasks-download 
depends=init,maven-ant-tasks-localrepo unless=maven-ant-tasks.jar.exists
 description=Fetch Maven ANT Tasks from Maven Central Repositroy
   echoDownloading Maven ANT Tasks.../echo
-  mkdir dir=${build.dir} /
   get 
src=${maven-ant-tasks.url}/${maven-ant-tasks.version}/maven-ant-tasks-${maven-ant-tasks.version}.jar
dest=${build.dir}/maven-ant-tasks-${maven-ant-tasks.version}.jar 
usetimestamp=true /
 /target
 
-target name=maven-ant-tasks-init depends=maven-ant-tasks-download 
unless=maven-ant-tasks.initialized
+target name=maven-ant-tasks-init 
depends=init,maven-ant-tasks-download unless=maven-ant-tasks.initialized
 description=Initialize Maven ANT Tasks
-  mkdir dir=${build.dir.lib}/
   typedef uri=antlib:org.apache.maven.artifact.ant 
classpathref=maven-ant-tasks.classpath /
 
   !-- define the remote repositories we use --
@@ -361,7 +363,7 @@
 exclusion groupId=org.apache.velocity artifactId=velocity/
   /dependency
   
-  dependency groupId=org.apache.thrift artifactId=libthrift 
version=0.7.0/
+  dependency groupId=org.apache.thrift artifactId=libthrift 
version=0.9.0/
 
   dependency groupId=com.thoughtworks.paranamer 
artifactId=paranamer-ant version=2.1/
   dependency groupId=junit artifactId=junit version=4.6 /
@@ -383,7 +385,7 @@
   dependency groupId=log4j artifactId=log4j version=1.2.16 /
   dependency groupId=org.apache.cassandra 
artifactId=cassandra-all version=${version} /
   dependency groupId=org.apache.cassandra 
artifactId=cassandra-thrift version=${version} /
-  dependency groupId=com.yammer.metrics artifactId=metrics-core 
version=2.0.3 /
+  dependency groupId=com.yammer.metrics artifactId=metrics-core 
version=2.2.0 /
   dependency groupId=edu.stanford.ppl artifactId=snaptree 
version=0.1 /
   dependency groupId=org.mindrot artifactId=jbcrypt 
version=0.3m /
   dependency groupId=io.netty artifactId=netty 
version=3.5.9.Final /
@@ -525,7 +527,7 @@
   /artifact:pom
 /target
 
-target name=maven-ant-tasks-retrieve-build 
depends=maven-declare-dependencies
+target name=maven-ant-tasks-retrieve-build 
depends=maven-declare-dependencies unless=without.maven
   artifact:dependencies pomRefId=build-deps-pom
  filesetId=build-dependency-jars 
  sourcesFilesetId=build-dependency-sources 

git commit: add support for bind variables to non-prepared statements.

2013-05-06 Thread marcuse
Updated Branches:
  refs/heads/trunk 3c06ff0a8 - df723af8a


add support for bind variables to non-prepared statements.

patch by marcuse, reviewed by pcmanus for CASSANDRA-5349


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

Branch: refs/heads/trunk
Commit: df723af8a6c04536abd2b4a48cd101f3b2e96746
Parents: 3c06ff0
Author: Marcus Eriksson marc...@spotify.com
Authored: Mon May 6 21:03:26 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Mon May 6 21:03:26 2013 +0200

--
 doc/native_protocol_v2.spec|   16 +-
 .../org/apache/cassandra/cql3/QueryProcessor.java  |   12 +++-
 .../apache/cassandra/transport/SimpleClient.java   |7 ++
 .../cassandra/transport/messages/QueryMessage.java |   44 +--
 4 files changed, 68 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/df723af8/doc/native_protocol_v2.spec
--
diff --git a/doc/native_protocol_v2.spec b/doc/native_protocol_v2.spec
index ad14929..a765700 100644
--- a/doc/native_protocol_v2.spec
+++ b/doc/native_protocol_v2.spec
@@ -263,8 +263,13 @@ Table of Contents
 
 4.1.4. QUERY
 
-  Performs a CQL query. The body of the message consists of a CQL query as a 
[long
-  string] followed by the [consistency] for the operation.
+  Performs a CQL query. The body of the message must be:
+queryconsistency[nvalue_1...value_n]
+  where:
+- query the query, [long string].
+- consistency is the [consistency] level for the operation.
+- optional: n [short], the number of following values.
+- optional: value_1...value_n are [bytes] to use for bound variables 
in the query.
 
   Note that the consistency is ignored by some queries (USE, CREATE, ALTER,
   TRUNCATE, ...).
@@ -638,3 +643,10 @@ Table of Contents
   executed if the provide prepared statement ID is not known by
   this host. The rest of the ERROR message body will be [short
   bytes] representing the unknown ID.
+
+8. Changes from v1
+  * Protocol is versioned to allow old client connects to a newer server, if a 
newer
+client connects to an older server, it needs to check if it gets a
+ProtocolException on connection and try connecting with a lower version.
+  * A query can now have bind variables even though the statement is not
+prepared. (see Section 4.1.4)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/df723af8/src/java/org/apache/cassandra/cql3/QueryProcessor.java
--
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java 
b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index 61b0b50..f7aebff 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -134,11 +134,17 @@ public class QueryProcessor
 public static ResultMessage process(String queryString, ConsistencyLevel 
cl, QueryState queryState)
 throws RequestExecutionException, RequestValidationException
 {
+return process(queryString, Collections.ByteBufferemptyList(), cl, 
queryState);
+}
+
+public static ResultMessage process(String queryString, ListByteBuffer 
variables, ConsistencyLevel cl, QueryState queryState)
+throws RequestExecutionException, RequestValidationException
+{
 logger.trace(CQL QUERY: {}, queryString);
 CQLStatement prepared = getStatement(queryString, 
queryState.getClientState()).statement;
-if (prepared.getBoundsTerms()  0)
-throw new InvalidRequestException(Cannot execute query with bind 
variables);
-return processStatement(prepared, cl, queryState, 
Collections.ByteBufferemptyList());
+if (prepared.getBoundsTerms() != variables.size())
+throw new InvalidRequestException(Invalid amount of bind 
variables);
+return processStatement(prepared, cl, queryState, variables);
 }
 
 public static UntypedResultSet process(String query, ConsistencyLevel cl) 
throws RequestExecutionException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/df723af8/src/java/org/apache/cassandra/transport/SimpleClient.java
--
diff --git a/src/java/org/apache/cassandra/transport/SimpleClient.java 
b/src/java/org/apache/cassandra/transport/SimpleClient.java
index 4caac61..993a490 100644
--- a/src/java/org/apache/cassandra/transport/SimpleClient.java
+++ b/src/java/org/apache/cassandra/transport

git commit: Dont throw UOE when calling shouldInclude in NamesQueryFilter - include all files instead

2013-05-24 Thread marcuse
Updated Branches:
  refs/heads/trunk 0ad499e9a - d106950ec


Dont throw UOE when calling shouldInclude in NamesQueryFilter - include all 
files instead


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

Branch: refs/heads/trunk
Commit: d106950ec049fc7dfef7e64b124dc8c050b06af4
Parents: 0ad499e
Author: Marcus Eriksson marc...@spotify.com
Authored: Fri May 24 18:54:59 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Fri May 24 18:54:59 2013 +0200

--
 .../cassandra/db/filter/NamesQueryFilter.java  |3 +--
 1 files changed, 1 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d106950e/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
--
diff --git a/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java 
b/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
index caddb0e..570eb29 100644
--- a/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
@@ -144,8 +144,7 @@ public class NamesQueryFilter implements IDiskAtomFilter
 
 public boolean shouldInclude(SSTableReader sstable)
 {
-// only called by collationcontroller for slice queries
-throw new UnsupportedOperationException();
+return true;
 }
 
 public static class Serializer implements 
IVersionedSerializerNamesQueryFilter



git commit: fix bug in intersection checking, and improve collection of max/min column

2013-05-31 Thread marcuse
Updated Branches:
  refs/heads/trunk 496035585 - 1ea5059fe


fix bug in intersection checking, and improve collection of max/min column

Patch by marcuse, reviewed by pcmanus for CASSANDRA-5600


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

Branch: refs/heads/trunk
Commit: 1ea5059fe3976ce8f660b46520859c31bb433fda
Parents: 4960355
Author: Marcus Eriksson marc...@spotify.com
Authored: Fri May 31 16:20:54 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Fri May 31 16:22:17 2013 +0200

--
 CHANGES.txt|2 +-
 .../apache/cassandra/db/marshal/CompositeType.java |   26 +---
 .../cassandra/io/sstable/ColumnNameHelper.java |  105 ++-
 3 files changed, 78 insertions(+), 55 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1ea5059f/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index abde4b3..eec5ea7 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -51,7 +51,7 @@
  * Use range tombstones when dropping cfs/columns from schema (CASSANDRA-5579)
  * cqlsh: drop CQL2/CQL3-beta support (CASSANDRA-5585)
  * Track max/min column names in sstables to be able to optimize slice
-   queries (CASSANDRA-5514, CASSANDRA-5595)
+   queries (CASSANDRA-5514, CASSANDRA-5595, CASSANDRA-5600)
  * Binary protocol: allow batching already prepared statements (CASSANDRA-4693)
  * Allow preparing timestamp, ttl and limit in CQL3 queries (CASSANDRA-4450)
  * Support native link w/o JNA in Java7 (CASSANDRA-3734)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1ea5059f/src/java/org/apache/cassandra/db/marshal/CompositeType.java
--
diff --git a/src/java/org/apache/cassandra/db/marshal/CompositeType.java 
b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
index ccc3b20..7679907 100644
--- a/src/java/org/apache/cassandra/db/marshal/CompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
@@ -191,32 +191,20 @@ public class CompositeType extends AbstractCompositeType
 return true;
 }
 
-/**
- * Deconstructs the composite and fills out any missing components with 
EMPTY_BYTE_BUFFER.
- */
-public ListAbstractCompositeType.CompositeComponent 
deconstructAndExpand(ByteBuffer composite)
-{
-ListAbstractCompositeType.CompositeComponent components = 
deconstruct(composite);
-for (int i = components.size(); i  types.size(); i++)
-components.add(new 
AbstractCompositeType.CompositeComponent(this.types.get(i), 
ByteBufferUtil.EMPTY_BYTE_BUFFER));
-return components;
-}
-
 @Override
 public boolean intersects(ListByteBuffer minColumnNames, 
ListByteBuffer maxColumnNames, SliceQueryFilter filter)
 {
-int typeCount = types.get(types.size() - 1) instanceof 
ColumnToCollectionType ? types.size() - 1 : types.size();
-
-assert minColumnNames.size() == typeCount;
-
+assert minColumnNames.size() == maxColumnNames.size();
 for (ColumnSlice slice : filter.slices)
 {
-ListAbstractCompositeType.CompositeComponent start = 
deconstructAndExpand(filter.isReversed() ? slice.finish : slice.start);
-ListAbstractCompositeType.CompositeComponent finish = 
deconstructAndExpand(filter.isReversed() ? slice.start : slice.finish);
-for (int i = 0; i  typeCount; i++)
+ByteBuffer[] start = split(filter.isReversed() ? slice.finish : 
slice.start);
+ByteBuffer[] finish = split(filter.isReversed() ? slice.start : 
slice.finish);
+for (int i = 0; i  minColumnNames.size(); i++)
 {
 AbstractType? t = types.get(i);
-if (!t.intersects(minColumnNames.get(i), 
maxColumnNames.get(i), start.get(i).value, finish.get(i).value))
+ByteBuffer s = i  start.length ? start[i] : 
ByteBufferUtil.EMPTY_BYTE_BUFFER;
+ByteBuffer f = i  finish.length ? finish[i] : 
ByteBufferUtil.EMPTY_BYTE_BUFFER;
+if (!t.intersects(minColumnNames.get(i), 
maxColumnNames.get(i), s, f))
 return false;
 }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1ea5059f/src/java/org/apache/cassandra/io/sstable/ColumnNameHelper.java
--
diff --git a/src/java/org/apache/cassandra/io/sstable/ColumnNameHelper.java 
b/src/java/org/apache/cassandra/io/sstable/ColumnNameHelper.java
index d9dc4b8..2ad1cff

git commit: another try at fixing the broken testMutateLevel test

2013-06-18 Thread marcuse
Updated Branches:
  refs/heads/trunk 8b908c0ac - 62295f68c


another try at fixing the broken testMutateLevel test


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

Branch: refs/heads/trunk
Commit: 62295f68c7b7b10cc7d41d72f0817e17133a9b36
Parents: 8b908c0
Author: Marcus Eriksson marc...@spotify.com
Authored: Tue Jun 18 17:57:22 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Tue Jun 18 17:57:22 2013 +0200

--
 .../cassandra/db/compaction/LeveledCompactionStrategyTest.java   | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/62295f68/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
--
diff --git 
a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
 
b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
index b33defc..d332ec3 100644
--- 
a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
+++ 
b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.db.compaction;
 
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
@@ -173,10 +174,11 @@ public class LeveledCompactionStrategyTest extends 
SchemaLoader
 cfs.forceBlockingFlush();
 }
 waitForLeveling(cfs);
+cfs.forceBlockingFlush();
 LeveledCompactionStrategy strategy = (LeveledCompactionStrategy) 
cfs.getCompactionStrategy();
 cfs.disableAutoCompaction();
 
-while(CompactionManager.instance.getActiveCompactions()  0)
+while(CompactionManager.instance.isCompacting(Arrays.asList(cfs)))
 Thread.sleep(100);
 
 for (SSTableReader s : cfs.getSSTables())



git commit: Don't keep ancestor information in memory since it is only used on startup.

2013-07-08 Thread marcuse
Updated Branches:
  refs/heads/cassandra-1.2 cc9ac8277 - 83b75754f


Don't keep ancestor information in memory since it is only used on startup.

patch by marcuse, reviewed by jbellis for CASSANDRA-5342


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

Branch: refs/heads/cassandra-1.2
Commit: 83b75754ff143d4d77b01ef76a813da47779c6f4
Parents: cc9ac82
Author: Marcus Eriksson marc...@spotify.com
Authored: Mon Jul 8 09:55:34 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Mon Jul 8 09:57:11 2013 +0200

--
 CHANGES.txt |  1 +
 .../cassandra/io/sstable/SSTableMetadata.java   | 39 ++--
 .../cassandra/io/sstable/SSTableReader.java | 12 +-
 .../cassandra/io/sstable/SSTableWriter.java |  6 +--
 .../cassandra/tools/SSTableMetadataViewer.java  |  2 +-
 .../sstable/SSTableMetadataSerializerTest.java  | 13 +--
 6 files changed, 52 insertions(+), 21 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/83b75754/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index fa4df36..b08f967 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@
(CASSANDRA-5692)
  * Fix skipping range tombstones with reverse queries (CASSANDRA-5712)
  * Expire entries out of ThriftSessionManager (CASSANRDA-5719)
+ * Don't keep ancestor information in memory (CASSANDRA-5342)
 
 1.2.6
  * Fix tracing when operation completes before all responses arrive 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/83b75754/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
--
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java 
b/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
index 865c2ea..de51ea9 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.io.sstable;
 import java.io.*;
 import java.util.*;
 
+import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.StreamingHistogram;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -55,7 +56,6 @@ public class SSTableMetadata
 public final long maxTimestamp;
 public final double compressionRatio;
 public final String partitioner;
-public final SetInteger ancestors;
 public final StreamingHistogram estimatedTombstoneDropTime;
 
 private SSTableMetadata()
@@ -67,7 +67,6 @@ public class SSTableMetadata
  Long.MAX_VALUE,
  NO_COMPRESSION_RATIO,
  null,
- Collections.IntegeremptySet(),
  defaultTombstoneDropTimeHistogram());
 }
 
@@ -78,7 +77,6 @@ public class SSTableMetadata
 long maxTimestamp,
 double cr,
 String partitioner,
-SetInteger ancestors,
 StreamingHistogram estimatedTombstoneDropTime)
 {
 this.estimatedRowSize = rowSizes;
@@ -88,7 +86,6 @@ public class SSTableMetadata
 this.maxTimestamp = maxTimestamp;
 this.compressionRatio = cr;
 this.partitioner = partitioner;
-this.ancestors = ancestors;
 this.estimatedTombstoneDropTime = estimatedTombstoneDropTime;
 }
 
@@ -193,7 +190,6 @@ public class SSTableMetadata
maxTimestamp,
compressionRatio,
partitioner,
-   ancestors,
estimatedTombstoneDropTime);
 }
 
@@ -242,7 +238,7 @@ public class SSTableMetadata
 {
 private static final Logger logger = 
LoggerFactory.getLogger(SSTableMetadataSerializer.class);
 
-public void serialize(SSTableMetadata sstableStats, DataOutput dos) 
throws IOException
+public void serialize(SSTableMetadata sstableStats, SetInteger 
ancestors, DataOutput dos) throws IOException
 {
 assert sstableStats.partitioner != null;
 
@@ -253,20 +249,31 @@ public class SSTableMetadata
 dos.writeLong(sstableStats.maxTimestamp);
 dos.writeDouble(sstableStats.compressionRatio);
 dos.writeUTF(sstableStats.partitioner);
-dos.writeInt(sstableStats.ancestors.size());
-for (Integer g : sstableStats.ancestors)
+dos.writeInt(ancestors.size

[1/2] git commit: Don't keep ancestor information in memory since it is only used on startup.

2013-07-08 Thread marcuse
Updated Branches:
  refs/heads/trunk 1a2f35731 - 07ca445e5


Don't keep ancestor information in memory since it is only used on startup.

patch by marcuse, reviewed by jbellis for CASSANDRA-5342


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

Branch: refs/heads/trunk
Commit: 83b75754ff143d4d77b01ef76a813da47779c6f4
Parents: cc9ac82
Author: Marcus Eriksson marc...@spotify.com
Authored: Mon Jul 8 09:55:34 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Mon Jul 8 09:57:11 2013 +0200

--
 CHANGES.txt |  1 +
 .../cassandra/io/sstable/SSTableMetadata.java   | 39 ++--
 .../cassandra/io/sstable/SSTableReader.java | 12 +-
 .../cassandra/io/sstable/SSTableWriter.java |  6 +--
 .../cassandra/tools/SSTableMetadataViewer.java  |  2 +-
 .../sstable/SSTableMetadataSerializerTest.java  | 13 +--
 6 files changed, 52 insertions(+), 21 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/83b75754/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index fa4df36..b08f967 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@
(CASSANDRA-5692)
  * Fix skipping range tombstones with reverse queries (CASSANDRA-5712)
  * Expire entries out of ThriftSessionManager (CASSANRDA-5719)
+ * Don't keep ancestor information in memory (CASSANDRA-5342)
 
 1.2.6
  * Fix tracing when operation completes before all responses arrive 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/83b75754/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
--
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java 
b/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
index 865c2ea..de51ea9 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.io.sstable;
 import java.io.*;
 import java.util.*;
 
+import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.StreamingHistogram;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -55,7 +56,6 @@ public class SSTableMetadata
 public final long maxTimestamp;
 public final double compressionRatio;
 public final String partitioner;
-public final SetInteger ancestors;
 public final StreamingHistogram estimatedTombstoneDropTime;
 
 private SSTableMetadata()
@@ -67,7 +67,6 @@ public class SSTableMetadata
  Long.MAX_VALUE,
  NO_COMPRESSION_RATIO,
  null,
- Collections.IntegeremptySet(),
  defaultTombstoneDropTimeHistogram());
 }
 
@@ -78,7 +77,6 @@ public class SSTableMetadata
 long maxTimestamp,
 double cr,
 String partitioner,
-SetInteger ancestors,
 StreamingHistogram estimatedTombstoneDropTime)
 {
 this.estimatedRowSize = rowSizes;
@@ -88,7 +86,6 @@ public class SSTableMetadata
 this.maxTimestamp = maxTimestamp;
 this.compressionRatio = cr;
 this.partitioner = partitioner;
-this.ancestors = ancestors;
 this.estimatedTombstoneDropTime = estimatedTombstoneDropTime;
 }
 
@@ -193,7 +190,6 @@ public class SSTableMetadata
maxTimestamp,
compressionRatio,
partitioner,
-   ancestors,
estimatedTombstoneDropTime);
 }
 
@@ -242,7 +238,7 @@ public class SSTableMetadata
 {
 private static final Logger logger = 
LoggerFactory.getLogger(SSTableMetadataSerializer.class);
 
-public void serialize(SSTableMetadata sstableStats, DataOutput dos) 
throws IOException
+public void serialize(SSTableMetadata sstableStats, SetInteger 
ancestors, DataOutput dos) throws IOException
 {
 assert sstableStats.partitioner != null;
 
@@ -253,20 +249,31 @@ public class SSTableMetadata
 dos.writeLong(sstableStats.maxTimestamp);
 dos.writeDouble(sstableStats.compressionRatio);
 dos.writeUTF(sstableStats.partitioner);
-dos.writeInt(sstableStats.ancestors.size());
-for (Integer g : sstableStats.ancestors)
+dos.writeInt(ancestors.size

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

2013-10-04 Thread marcuse
Merge branch 'cassandra-2.0' into trunk


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

Branch: refs/heads/trunk
Commit: bbf3cc5166a8279db851090e64997ffc00cb8f15
Parents: 6dda6f2 8dfd75d
Author: Marcus Eriksson marc...@spotify.com
Authored: Fri Oct 4 09:42:13 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Fri Oct 4 09:42:13 2013 +0200

--
 CHANGES.txt |  1 +
 .../db/compaction/LeveledManifest.java  |  4 ++
 .../org/apache/cassandra/io/util/FileUtils.java | 57 
 .../utils/BackgroundActivityMonitor.java| 10 +---
 .../org/apache/cassandra/utils/CLibrary.java|  2 -
 .../org/apache/cassandra/utils/FBUtilities.java |  7 +++
 6 files changed, 62 insertions(+), 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/bbf3cc51/CHANGES.txt
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bbf3cc51/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
--



[1/2] git commit: Use Java7 file APIs to get saner exceptions etc.

2013-10-04 Thread marcuse
Updated Branches:
  refs/heads/trunk 6dda6f2d8 - bbf3cc516


Use Java7 file APIs to get saner exceptions etc.

And since windows does not support moving a file on top of another
existing file, remove the old file before moving.

Patch by marcuse, reviewed by jbellis for CASSANDRA-5383.


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

Branch: refs/heads/trunk
Commit: 8dfd75de160a29516af315166649d5c66cec3113
Parents: 213f6bb
Author: Marcus Eriksson marc...@spotify.com
Authored: Fri Oct 4 09:37:22 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Fri Oct 4 09:37:22 2013 +0200

--
 CHANGES.txt |  1 +
 .../db/compaction/LeveledManifest.java  |  4 ++
 .../org/apache/cassandra/io/util/FileUtils.java | 57 
 .../utils/BackgroundActivityMonitor.java| 10 +---
 .../org/apache/cassandra/utils/CLibrary.java|  2 -
 .../org/apache/cassandra/utils/FBUtilities.java |  7 +++
 6 files changed, 62 insertions(+), 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8dfd75de/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index e08e5b6..acd4d52 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -16,6 +16,7 @@
  * Delete can potentially be skipped in batch (CASSANDRA-6115)
  * Allow alter keyspace on system_traces (CASSANDRA-6016)
  * Disallow empty column names in cql (CASSANDRA-6136)
+ * Use Java7 file-handling APIs and fix file moving on Windows (CASSANDRA-5383)
 Merged from 1.2:
  * Never return WriteTimeout for CL.ANY (CASSANDRA-6032)
  * Tracing should log write failure rather than raw exceptions (CASSANDRA-6133)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8dfd75de/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
--
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java 
b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
index 82aa2d6..7348c29 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
@@ -39,6 +39,7 @@ import org.apache.cassandra.dht.Bounds;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 
 public class LeveledManifest
@@ -595,6 +596,9 @@ public class LeveledManifest
 SSTableMetadata.serializer.legacySerialize(metadata, 
oldMetadata.right, descriptor, out);
 out.flush();
 out.close();
+// we cant move a file on top of another file in windows:
+if (!FBUtilities.isUnix())
+FileUtils.delete(filename);
 FileUtils.renameWithConfirm(filename + -tmp, filename);
 }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8dfd75de/src/java/org/apache/cassandra/io/util/FileUtils.java
--
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java 
b/src/java/org/apache/cassandra/io/util/FileUtils.java
index a0be773..6b91bd3 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -22,7 +22,9 @@ import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.nio.MappedByteBuffer;
 import java.nio.file.Files;
-import java.nio.file.Paths;
+import java.nio.file.AtomicMoveNotSupportedException;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
 import java.text.DecimalFormat;
 import java.util.Arrays;
 
@@ -74,9 +76,7 @@ public class FileUtils
 
 try
 {
-// Avoiding getAbsolutePath() in case there is ever a difference 
between that and the the
-// behavior of nio2.
-Files.createLink(Paths.get(to.getPath()), 
Paths.get(from.getPath()));
+Files.createLink(to.toPath(), from.toPath());
 }
 catch (IOException e)
 {
@@ -111,13 +111,27 @@ public class FileUtils
 assert file.exists() : attempted to delete non-existing file  + 
file.getName();
 if (logger.isDebugEnabled())
 logger.debug(Deleting  + file.getName());
-if (!file.delete())
-throw new FSWriteError(new IOException(Failed to delete  + 
file.getAbsolutePath()), file);
+try
+{
+Files.delete(file.toPath

git commit: Use Java7 file APIs to get saner exceptions etc.

2013-10-04 Thread marcuse
Updated Branches:
  refs/heads/cassandra-2.0 213f6bb88 - 8dfd75de1


Use Java7 file APIs to get saner exceptions etc.

And since windows does not support moving a file on top of another
existing file, remove the old file before moving.

Patch by marcuse, reviewed by jbellis for CASSANDRA-5383.


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

Branch: refs/heads/cassandra-2.0
Commit: 8dfd75de160a29516af315166649d5c66cec3113
Parents: 213f6bb
Author: Marcus Eriksson marc...@spotify.com
Authored: Fri Oct 4 09:37:22 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Fri Oct 4 09:37:22 2013 +0200

--
 CHANGES.txt |  1 +
 .../db/compaction/LeveledManifest.java  |  4 ++
 .../org/apache/cassandra/io/util/FileUtils.java | 57 
 .../utils/BackgroundActivityMonitor.java| 10 +---
 .../org/apache/cassandra/utils/CLibrary.java|  2 -
 .../org/apache/cassandra/utils/FBUtilities.java |  7 +++
 6 files changed, 62 insertions(+), 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8dfd75de/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index e08e5b6..acd4d52 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -16,6 +16,7 @@
  * Delete can potentially be skipped in batch (CASSANDRA-6115)
  * Allow alter keyspace on system_traces (CASSANDRA-6016)
  * Disallow empty column names in cql (CASSANDRA-6136)
+ * Use Java7 file-handling APIs and fix file moving on Windows (CASSANDRA-5383)
 Merged from 1.2:
  * Never return WriteTimeout for CL.ANY (CASSANDRA-6032)
  * Tracing should log write failure rather than raw exceptions (CASSANDRA-6133)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8dfd75de/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
--
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java 
b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
index 82aa2d6..7348c29 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
@@ -39,6 +39,7 @@ import org.apache.cassandra.dht.Bounds;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 
 public class LeveledManifest
@@ -595,6 +596,9 @@ public class LeveledManifest
 SSTableMetadata.serializer.legacySerialize(metadata, 
oldMetadata.right, descriptor, out);
 out.flush();
 out.close();
+// we cant move a file on top of another file in windows:
+if (!FBUtilities.isUnix())
+FileUtils.delete(filename);
 FileUtils.renameWithConfirm(filename + -tmp, filename);
 }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8dfd75de/src/java/org/apache/cassandra/io/util/FileUtils.java
--
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java 
b/src/java/org/apache/cassandra/io/util/FileUtils.java
index a0be773..6b91bd3 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -22,7 +22,9 @@ import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.nio.MappedByteBuffer;
 import java.nio.file.Files;
-import java.nio.file.Paths;
+import java.nio.file.AtomicMoveNotSupportedException;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
 import java.text.DecimalFormat;
 import java.util.Arrays;
 
@@ -74,9 +76,7 @@ public class FileUtils
 
 try
 {
-// Avoiding getAbsolutePath() in case there is ever a difference 
between that and the the
-// behavior of nio2.
-Files.createLink(Paths.get(to.getPath()), 
Paths.get(from.getPath()));
+Files.createLink(to.toPath(), from.toPath());
 }
 catch (IOException e)
 {
@@ -111,13 +111,27 @@ public class FileUtils
 assert file.exists() : attempted to delete non-existing file  + 
file.getName();
 if (logger.isDebugEnabled())
 logger.debug(Deleting  + file.getName());
-if (!file.delete())
-throw new FSWriteError(new IOException(Failed to delete  + 
file.getAbsolutePath()), file);
+try
+{
+Files.delete(file.toPath

git commit: fix merge

2013-10-18 Thread marcuse
Updated Branches:
  refs/heads/trunk a9d64ef86 - 6bcf28edc


fix merge


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

Branch: refs/heads/trunk
Commit: 6bcf28edcb1d59253113102efd7ac25ffb35037f
Parents: a9d64ef
Author: Marcus Eriksson marc...@spotify.com
Authored: Fri Oct 18 15:28:21 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Fri Oct 18 15:28:21 2013 +0200

--
 src/java/org/apache/cassandra/service/StorageService.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bcf28ed/src/java/org/apache/cassandra/service/StorageService.java
--
diff --git a/src/java/org/apache/cassandra/service/StorageService.java 
b/src/java/org/apache/cassandra/service/StorageService.java
index cd09083..22a9270 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -582,7 +582,7 @@ public class StorageService extends 
NotificationBroadcasterSupport implements IE
 // for bootstrap to get the load info it needs.
 // (we won't be part of the storage ring though until we add a 
counterId to our state, below.)
 // Seed the host ID-to-endpoint map with our own ID.
-getTokenMetadata().updateHostId(SystemTable.getLocalHostId(), 
FBUtilities.getBroadcastAddress());
+getTokenMetadata().updateHostId(SystemKeyspace.getLocalHostId(), 
FBUtilities.getBroadcastAddress());
 appStates.put(ApplicationState.NET_VERSION, 
valueFactory.networkVersion());
 appStates.put(ApplicationState.HOST_ID, 
valueFactory.hostId(SystemKeyspace.getLocalHostId()));
 appStates.put(ApplicationState.RPC_ADDRESS, 
valueFactory.rpcaddress(DatabaseDescriptor.getRpcAddress()));



[1/2] git commit: really fix build this time

2013-10-18 Thread marcuse
Updated Branches:
  refs/heads/trunk 6bcf28edc - 3f8a90aa4


really fix build this time


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

Branch: refs/heads/trunk
Commit: 91bd207ffb3bb8fb1071488a372f65ea168cf222
Parents: 4dc5d69
Author: Marcus Eriksson marc...@spotify.com
Authored: Fri Oct 18 16:08:20 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Fri Oct 18 16:08:20 2013 +0200

--
 src/java/org/apache/cassandra/service/StorageService.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/91bd207f/src/java/org/apache/cassandra/service/StorageService.java
--
diff --git a/src/java/org/apache/cassandra/service/StorageService.java 
b/src/java/org/apache/cassandra/service/StorageService.java
index e4e03a2..b158a81 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -585,7 +585,7 @@ public class StorageService extends 
NotificationBroadcasterSupport implements IE
 // for bootstrap to get the load info it needs.
 // (we won't be part of the storage ring though until we add a 
counterId to our state, below.)
 // Seed the host ID-to-endpoint map with our own ID.
-getTokenMetadata().updateHostId(SystemTable.getLocalHostId(), 
FBUtilities.getBroadcastAddress());
+getTokenMetadata().updateHostId(SystemKeyspace.getLocalHostId(), 
FBUtilities.getBroadcastAddress());
 appStates.put(ApplicationState.NET_VERSION, 
valueFactory.networkVersion());
 appStates.put(ApplicationState.HOST_ID, 
valueFactory.hostId(SystemKeyspace.getLocalHostId()));
 appStates.put(ApplicationState.RPC_ADDRESS, 
valueFactory.rpcaddress(DatabaseDescriptor.getRpcAddress()));



git commit: really fix build this time

2013-10-18 Thread marcuse
Updated Branches:
  refs/heads/cassandra-2.0 4dc5d690f - 91bd207ff


really fix build this time


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

Branch: refs/heads/cassandra-2.0
Commit: 91bd207ffb3bb8fb1071488a372f65ea168cf222
Parents: 4dc5d69
Author: Marcus Eriksson marc...@spotify.com
Authored: Fri Oct 18 16:08:20 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Fri Oct 18 16:08:20 2013 +0200

--
 src/java/org/apache/cassandra/service/StorageService.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/91bd207f/src/java/org/apache/cassandra/service/StorageService.java
--
diff --git a/src/java/org/apache/cassandra/service/StorageService.java 
b/src/java/org/apache/cassandra/service/StorageService.java
index e4e03a2..b158a81 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -585,7 +585,7 @@ public class StorageService extends 
NotificationBroadcasterSupport implements IE
 // for bootstrap to get the load info it needs.
 // (we won't be part of the storage ring though until we add a 
counterId to our state, below.)
 // Seed the host ID-to-endpoint map with our own ID.
-getTokenMetadata().updateHostId(SystemTable.getLocalHostId(), 
FBUtilities.getBroadcastAddress());
+getTokenMetadata().updateHostId(SystemKeyspace.getLocalHostId(), 
FBUtilities.getBroadcastAddress());
 appStates.put(ApplicationState.NET_VERSION, 
valueFactory.networkVersion());
 appStates.put(ApplicationState.HOST_ID, 
valueFactory.hostId(SystemKeyspace.getLocalHostId()));
 appStates.put(ApplicationState.RPC_ADDRESS, 
valueFactory.rpcaddress(DatabaseDescriptor.getRpcAddress()));



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

2013-10-18 Thread marcuse
Merge branch 'cassandra-2.0' into trunk


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

Branch: refs/heads/trunk
Commit: 3f8a90aa41709eb9070002a9b2e8ebd2306c4d26
Parents: 6bcf28e 91bd207
Author: Marcus Eriksson marc...@spotify.com
Authored: Fri Oct 18 16:08:25 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Fri Oct 18 16:08:25 2013 +0200

--

--




git commit: Revert remove decompression code in CompressedSequentialWriter

2013-10-19 Thread marcuse
Updated Branches:
  refs/heads/cassandra-2.0 cfa0e580e - 232906d7e


Revert remove decompression code in CompressedSequentialWriter

This reverts commit 9644f098967d6ba9154725148a383b3502ef62c2.


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

Branch: refs/heads/cassandra-2.0
Commit: 232906d7e01587c32739c61cd72079715c0abb3c
Parents: cfa0e58
Author: Marcus Eriksson marc...@spotify.com
Authored: Sat Oct 19 08:48:43 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Sat Oct 19 08:50:53 2013 +0200

--
 .../io/compress/CompressedSequentialWriter.java | 12 
 1 file changed, 12 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/232906d7/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
--
diff --git 
a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java 
b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index da4307e..54b990f 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -21,6 +21,7 @@ import java.io.EOFException;
 import java.io.File;
 import java.io.IOException;
 import java.util.zip.Adler32;
+import java.util.zip.CRC32;
 import java.util.zip.Checksum;
 
 import org.apache.cassandra.io.FSReadError;
@@ -192,6 +193,17 @@ public class CompressedSequentialWriter extends 
SequentialWriter
 {
 out.seek(chunkOffset);
 out.readFully(compressed.buffer, 0, chunkSize);
+
+try
+{
+// repopulate buffer
+compressor.uncompress(compressed.buffer, 0, chunkSize, buffer, 
0);
+}
+catch (IOException e)
+{
+throw new CorruptBlockException(getPath(), chunkOffset, 
chunkSize);
+}
+
 checksum.update(compressed.buffer, 0, chunkSize);
 
 if (out.readInt() != (int) checksum.getValue())



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

2013-10-19 Thread marcuse
Merge branch 'cassandra-2.0' into trunk


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

Branch: refs/heads/trunk
Commit: 656d3240f7488be7cbc05fab7e9dd45616c5cd27
Parents: 3020555 232906d
Author: Marcus Eriksson marc...@spotify.com
Authored: Sat Oct 19 08:51:54 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Sat Oct 19 08:51:54 2013 +0200

--
 .../io/compress/CompressedSequentialWriter.java | 12 
 1 file changed, 12 insertions(+)
--




[1/2] git commit: Revert remove decompression code in CompressedSequentialWriter

2013-10-19 Thread marcuse
Updated Branches:
  refs/heads/trunk 302055583 - 656d3240f


Revert remove decompression code in CompressedSequentialWriter

This reverts commit 9644f098967d6ba9154725148a383b3502ef62c2.


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

Branch: refs/heads/trunk
Commit: 232906d7e01587c32739c61cd72079715c0abb3c
Parents: cfa0e58
Author: Marcus Eriksson marc...@spotify.com
Authored: Sat Oct 19 08:48:43 2013 +0200
Committer: Marcus Eriksson marc...@spotify.com
Committed: Sat Oct 19 08:50:53 2013 +0200

--
 .../io/compress/CompressedSequentialWriter.java | 12 
 1 file changed, 12 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/232906d7/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
--
diff --git 
a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java 
b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index da4307e..54b990f 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -21,6 +21,7 @@ import java.io.EOFException;
 import java.io.File;
 import java.io.IOException;
 import java.util.zip.Adler32;
+import java.util.zip.CRC32;
 import java.util.zip.Checksum;
 
 import org.apache.cassandra.io.FSReadError;
@@ -192,6 +193,17 @@ public class CompressedSequentialWriter extends 
SequentialWriter
 {
 out.seek(chunkOffset);
 out.readFully(compressed.buffer, 0, chunkSize);
+
+try
+{
+// repopulate buffer
+compressor.uncompress(compressed.buffer, 0, chunkSize, buffer, 
0);
+}
+catch (IOException e)
+{
+throw new CorruptBlockException(getPath(), chunkOffset, 
chunkSize);
+}
+
 checksum.update(compressed.buffer, 0, chunkSize);
 
 if (out.readInt() != (int) checksum.getValue())



git commit: Use AtomicIntegerFieldUpdater in RefCountedMemory to save memory in row cache.

2013-11-01 Thread marcuse
Updated Branches:
  refs/heads/trunk 69fef1977 - 7918718c8


Use AtomicIntegerFieldUpdater in RefCountedMemory to save memory in row cache.

Patch by marcuse, reviewed by jbellis for CASSANDRA-6278


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

Branch: refs/heads/trunk
Commit: 7918718c8ce53c6d0958aa969fc813e586463d08
Parents: 69fef19
Author: Marcus Eriksson marc...@spotify.com
Authored: Fri Nov 1 06:57:48 2013 +0100
Committer: Marcus Eriksson marc...@spotify.com
Committed: Fri Nov 1 07:00:08 2013 +0100

--
 CHANGES.txt  |  2 +-
 .../org/apache/cassandra/cache/RefCountedMemory.java | 11 ++-
 2 files changed, 7 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7918718c/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index cc878e4..ad05970 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -6,7 +6,7 @@
  * Remove 1.2 network compatibility code (CASSANDRA-5960)
  * Remove leveled json manifest migration code (CASSANDRA-5996)
  * Remove CFDefinition (CASSANDRA-6253)
-
+ * Use AtomicIntegerFieldUpdater in RefCountedMemory (CASSANDRA-6278)
 
 2.0.3
  * Fix repair assertion error when tombstones expire (CASSANDRA-6277)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7918718c/src/java/org/apache/cassandra/cache/RefCountedMemory.java
--
diff --git a/src/java/org/apache/cassandra/cache/RefCountedMemory.java 
b/src/java/org/apache/cassandra/cache/RefCountedMemory.java
index 887bd86..76d9b00 100644
--- a/src/java/org/apache/cassandra/cache/RefCountedMemory.java
+++ b/src/java/org/apache/cassandra/cache/RefCountedMemory.java
@@ -17,13 +17,14 @@
  */
 package org.apache.cassandra.cache;
 
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 
 import org.apache.cassandra.io.util.Memory;
 
 public class RefCountedMemory extends Memory
 {
-private final AtomicInteger references = new AtomicInteger(1);
+private volatile int references = 1;
+private static final AtomicIntegerFieldUpdaterRefCountedMemory UPDATER = 
AtomicIntegerFieldUpdater.newUpdater(RefCountedMemory.class, references);
 
 public RefCountedMemory(long size)
 {
@@ -38,10 +39,10 @@ public class RefCountedMemory extends Memory
 {
 while (true)
 {
-int n = references.get();
+int n = UPDATER.get(this);
 if (n = 0)
 return false;
-if (references.compareAndSet(n, n + 1))
+if (UPDATER.compareAndSet(this, n, n + 1))
 return true;
 }
 }
@@ -49,7 +50,7 @@ public class RefCountedMemory extends Memory
 /** decrement reference count.  if count reaches zero, the object is 
freed. */
 public void unreference()
 {
-if (references.decrementAndGet() == 0)
+if (UPDATER.decrementAndGet(this) == 0)
 free();
 }
 }



git commit: Set minTimestamp correctly to be able to drop expired sstables correctly

2013-11-13 Thread marcuse
Updated Branches:
  refs/heads/cassandra-2.0 5008507ca - e02341b38


Set minTimestamp correctly to be able to drop expired sstables correctly

patch by marcuse, reviewed by jbellis for CASSANDRA-6337


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

Branch: refs/heads/cassandra-2.0
Commit: e02341b38c5c311bea2b0d954373ba55bb106446
Parents: 5008507
Author: Marcus Eriksson marc...@spotify.com
Authored: Wed Nov 13 20:12:51 2013 +0100
Committer: Marcus Eriksson marc...@spotify.com
Committed: Wed Nov 13 20:12:51 2013 +0100

--
 CHANGES.txt| 1 +
 .../org/apache/cassandra/db/compaction/CompactionController.java   | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e02341b3/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 159e8de..6a07720 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -21,6 +21,7 @@
  * Fix CQL3 table validation in Thrift (CASSANDRA-6140)
  * Fix bug missing results with IN clauses (CASSANDRA-6327)
  * Fix paging with reversed slices (CASSANDRA-6343)
+ * Set minTimestamp correctly to be able to drop expired sstables 
(CASSANDRA-6337)
 Merged from 1.2:
  * add non-jamm path for cached statements (CASSANDRA-6293)
  * (Hadoop) Require CFRR batchSize to be at least 2 (CASSANDRA-6114)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e02341b3/src/java/org/apache/cassandra/db/compaction/CompactionController.java
--
diff --git 
a/src/java/org/apache/cassandra/db/compaction/CompactionController.java 
b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
index 65515d6..3ab2202 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionController.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
@@ -106,7 +106,7 @@ public class CompactionController
 
 ListSSTableReader candidates = new ArrayListSSTableReader();
 
-long minTimestamp = Integer.MAX_VALUE;
+long minTimestamp = Long.MAX_VALUE;
 
 for (SSTableReader sstable : overlapping)
 minTimestamp = Math.min(minTimestamp, sstable.getMinTimestamp());



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

2013-11-13 Thread marcuse
Merge branch 'cassandra-2.0' into trunk


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

Branch: refs/heads/trunk
Commit: b92c55e68c005fe5581b50f585d9bc47dbdfaedb
Parents: cb871ba e02341b
Author: Marcus Eriksson marc...@spotify.com
Authored: Wed Nov 13 20:16:59 2013 +0100
Committer: Marcus Eriksson marc...@spotify.com
Committed: Wed Nov 13 20:16:59 2013 +0100

--
 CHANGES.txt| 1 +
 .../org/apache/cassandra/db/compaction/CompactionController.java   | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b92c55e6/CHANGES.txt
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b92c55e6/src/java/org/apache/cassandra/db/compaction/CompactionController.java
--



[1/2] git commit: Set minTimestamp correctly to be able to drop expired sstables correctly

2013-11-13 Thread marcuse
Updated Branches:
  refs/heads/trunk cb871ba90 - b92c55e68


Set minTimestamp correctly to be able to drop expired sstables correctly

patch by marcuse, reviewed by jbellis for CASSANDRA-6337


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

Branch: refs/heads/trunk
Commit: e02341b38c5c311bea2b0d954373ba55bb106446
Parents: 5008507
Author: Marcus Eriksson marc...@spotify.com
Authored: Wed Nov 13 20:12:51 2013 +0100
Committer: Marcus Eriksson marc...@spotify.com
Committed: Wed Nov 13 20:12:51 2013 +0100

--
 CHANGES.txt| 1 +
 .../org/apache/cassandra/db/compaction/CompactionController.java   | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e02341b3/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 159e8de..6a07720 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -21,6 +21,7 @@
  * Fix CQL3 table validation in Thrift (CASSANDRA-6140)
  * Fix bug missing results with IN clauses (CASSANDRA-6327)
  * Fix paging with reversed slices (CASSANDRA-6343)
+ * Set minTimestamp correctly to be able to drop expired sstables 
(CASSANDRA-6337)
 Merged from 1.2:
  * add non-jamm path for cached statements (CASSANDRA-6293)
  * (Hadoop) Require CFRR batchSize to be at least 2 (CASSANDRA-6114)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e02341b3/src/java/org/apache/cassandra/db/compaction/CompactionController.java
--
diff --git 
a/src/java/org/apache/cassandra/db/compaction/CompactionController.java 
b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
index 65515d6..3ab2202 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionController.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
@@ -106,7 +106,7 @@ public class CompactionController
 
 ListSSTableReader candidates = new ArrayListSSTableReader();
 
-long minTimestamp = Integer.MAX_VALUE;
+long minTimestamp = Long.MAX_VALUE;
 
 for (SSTableReader sstable : overlapping)
 minTimestamp = Math.min(minTimestamp, sstable.getMinTimestamp());



git commit: Fix CQLSSTableWriterTest

2013-11-27 Thread marcuse
Updated Branches:
  refs/heads/cassandra-2.0 2af076494 - 4fae76c14


Fix CQLSSTableWriterTest


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

Branch: refs/heads/cassandra-2.0
Commit: 4fae76c146f2856063927c9a8d1c34bd86b30a58
Parents: 2af0764
Author: Marcus Eriksson marc...@spotify.com
Authored: Wed Nov 27 18:58:47 2013 +0100
Committer: Marcus Eriksson marc...@spotify.com
Committed: Wed Nov 27 18:58:47 2013 +0100

--
 .../org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java| 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4fae76c1/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
--
diff --git 
a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java 
b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
index 7095b35..0e38e16 100644
--- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
@@ -39,7 +39,7 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.OutputHandler;
 
-public class CQLSSTableWriterTest extends SchemaLoader
+public class CQLSSTableWriterTest
 {
 @BeforeClass
 public static void setup() throws Exception
@@ -78,7 +78,7 @@ public class CQLSSTableWriterTest extends SchemaLoader
 {
 public void init(String keyspace)
 {
-for (RangeToken range : 
StorageService.instance.getLocalRanges(Keyspace1))
+for (RangeToken range : 
StorageService.instance.getLocalRanges(cql_keyspace))
 addRangeForEndpoint(range, 
FBUtilities.getBroadcastAddress());
 setPartitioner(StorageService.getPartitioner());
 }



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

2013-11-27 Thread marcuse
Merge branch 'cassandra-2.0' into trunk


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

Branch: refs/heads/trunk
Commit: f1a67a06a8fa3358b7ee4059f35e515f913cf197
Parents: ed0c91e 4fae76c
Author: Marcus Eriksson marc...@spotify.com
Authored: Wed Nov 27 18:58:55 2013 +0100
Committer: Marcus Eriksson marc...@spotify.com
Committed: Wed Nov 27 18:58:55 2013 +0100

--
 .../org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java| 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
--




[1/2] git commit: Fix CQLSSTableWriterTest

2013-11-27 Thread marcuse
Updated Branches:
  refs/heads/trunk ed0c91ee4 - f1a67a06a


Fix CQLSSTableWriterTest


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

Branch: refs/heads/trunk
Commit: 4fae76c146f2856063927c9a8d1c34bd86b30a58
Parents: 2af0764
Author: Marcus Eriksson marc...@spotify.com
Authored: Wed Nov 27 18:58:47 2013 +0100
Committer: Marcus Eriksson marc...@spotify.com
Committed: Wed Nov 27 18:58:47 2013 +0100

--
 .../org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java| 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4fae76c1/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
--
diff --git 
a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java 
b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
index 7095b35..0e38e16 100644
--- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
@@ -39,7 +39,7 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.OutputHandler;
 
-public class CQLSSTableWriterTest extends SchemaLoader
+public class CQLSSTableWriterTest
 {
 @BeforeClass
 public static void setup() throws Exception
@@ -78,7 +78,7 @@ public class CQLSSTableWriterTest extends SchemaLoader
 {
 public void init(String keyspace)
 {
-for (RangeToken range : 
StorageService.instance.getLocalRanges(Keyspace1))
+for (RangeToken range : 
StorageService.instance.getLocalRanges(cql_keyspace))
 addRangeForEndpoint(range, 
FBUtilities.getBroadcastAddress());
 setPartitioner(StorageService.getPartitioner());
 }



git commit: fix MoveTest

2013-12-04 Thread marcuse
Updated Branches:
  refs/heads/cassandra-1.2 f634ac7ea - d8c4e89b3


fix MoveTest


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

Branch: refs/heads/cassandra-1.2
Commit: d8c4e89b3e85e8cb41a438963845cb10a923a3d6
Parents: f634ac7
Author: Marcus Eriksson marc...@spotify.com
Authored: Wed Dec 4 20:17:30 2013 +0100
Committer: Marcus Eriksson marc...@spotify.com
Committed: Wed Dec 4 20:17:30 2013 +0100

--
 test/unit/org/apache/cassandra/service/MoveTest.java | 2 ++
 1 file changed, 2 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d8c4e89b/test/unit/org/apache/cassandra/service/MoveTest.java
--
diff --git a/test/unit/org/apache/cassandra/service/MoveTest.java 
b/test/unit/org/apache/cassandra/service/MoveTest.java
index e30bbde..07a9590 100644
--- a/test/unit/org/apache/cassandra/service/MoveTest.java
+++ b/test/unit/org/apache/cassandra/service/MoveTest.java
@@ -109,6 +109,7 @@ public class MoveTest
 
 // Third node leaves
 ss.onChange(hosts.get(MOVING_NODE), ApplicationState.STATUS, 
valueFactory.moving(newToken));
+PendingRangeCalculatorService.instance.blockUntilFinished();
 
 assertTrue(tmd.isMoving(hosts.get(MOVING_NODE)));
 
@@ -197,6 +198,7 @@ public class MoveTest
 ss.onChange(boot2,
 ApplicationState.STATUS,
 
valueFactory.bootstrapping(Collections.Tokensingleton(keyTokens.get(7;
+PendingRangeCalculatorService.instance.blockUntilFinished();
 
 // don't require test update every time a new keyspace is added to 
test/conf/cassandra.yaml
 MapString, AbstractReplicationStrategy tableStrategyMap = new 
HashMapString, AbstractReplicationStrategy();



[3/3] git commit: Merge branch 'cassandra-2.0' into trunk

2013-12-04 Thread marcuse
Merge branch 'cassandra-2.0' into trunk


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

Branch: refs/heads/trunk
Commit: e4d447240a4b68a56596445ac5e4f4fbbe0c50af
Parents: b34d43f 32dbe58
Author: Marcus Eriksson marc...@spotify.com
Authored: Wed Dec 4 20:17:45 2013 +0100
Committer: Marcus Eriksson marc...@spotify.com
Committed: Wed Dec 4 20:17:45 2013 +0100

--
 test/unit/org/apache/cassandra/service/MoveTest.java | 2 ++
 1 file changed, 2 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e4d44724/test/unit/org/apache/cassandra/service/MoveTest.java
--



[1/2] git commit: fix MoveTest

2013-12-04 Thread marcuse
Updated Branches:
  refs/heads/cassandra-2.0 1334f94e4 - 32dbe5825


fix MoveTest


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

Branch: refs/heads/cassandra-2.0
Commit: d8c4e89b3e85e8cb41a438963845cb10a923a3d6
Parents: f634ac7
Author: Marcus Eriksson marc...@spotify.com
Authored: Wed Dec 4 20:17:30 2013 +0100
Committer: Marcus Eriksson marc...@spotify.com
Committed: Wed Dec 4 20:17:30 2013 +0100

--
 test/unit/org/apache/cassandra/service/MoveTest.java | 2 ++
 1 file changed, 2 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d8c4e89b/test/unit/org/apache/cassandra/service/MoveTest.java
--
diff --git a/test/unit/org/apache/cassandra/service/MoveTest.java 
b/test/unit/org/apache/cassandra/service/MoveTest.java
index e30bbde..07a9590 100644
--- a/test/unit/org/apache/cassandra/service/MoveTest.java
+++ b/test/unit/org/apache/cassandra/service/MoveTest.java
@@ -109,6 +109,7 @@ public class MoveTest
 
 // Third node leaves
 ss.onChange(hosts.get(MOVING_NODE), ApplicationState.STATUS, 
valueFactory.moving(newToken));
+PendingRangeCalculatorService.instance.blockUntilFinished();
 
 assertTrue(tmd.isMoving(hosts.get(MOVING_NODE)));
 
@@ -197,6 +198,7 @@ public class MoveTest
 ss.onChange(boot2,
 ApplicationState.STATUS,
 
valueFactory.bootstrapping(Collections.Tokensingleton(keyTokens.get(7;
+PendingRangeCalculatorService.instance.blockUntilFinished();
 
 // don't require test update every time a new keyspace is added to 
test/conf/cassandra.yaml
 MapString, AbstractReplicationStrategy tableStrategyMap = new 
HashMapString, AbstractReplicationStrategy();



[1/3] git commit: fix MoveTest

2013-12-04 Thread marcuse
Updated Branches:
  refs/heads/trunk b34d43f97 - e4d447240


fix MoveTest


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

Branch: refs/heads/trunk
Commit: d8c4e89b3e85e8cb41a438963845cb10a923a3d6
Parents: f634ac7
Author: Marcus Eriksson marc...@spotify.com
Authored: Wed Dec 4 20:17:30 2013 +0100
Committer: Marcus Eriksson marc...@spotify.com
Committed: Wed Dec 4 20:17:30 2013 +0100

--
 test/unit/org/apache/cassandra/service/MoveTest.java | 2 ++
 1 file changed, 2 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d8c4e89b/test/unit/org/apache/cassandra/service/MoveTest.java
--
diff --git a/test/unit/org/apache/cassandra/service/MoveTest.java 
b/test/unit/org/apache/cassandra/service/MoveTest.java
index e30bbde..07a9590 100644
--- a/test/unit/org/apache/cassandra/service/MoveTest.java
+++ b/test/unit/org/apache/cassandra/service/MoveTest.java
@@ -109,6 +109,7 @@ public class MoveTest
 
 // Third node leaves
 ss.onChange(hosts.get(MOVING_NODE), ApplicationState.STATUS, 
valueFactory.moving(newToken));
+PendingRangeCalculatorService.instance.blockUntilFinished();
 
 assertTrue(tmd.isMoving(hosts.get(MOVING_NODE)));
 
@@ -197,6 +198,7 @@ public class MoveTest
 ss.onChange(boot2,
 ApplicationState.STATUS,
 
valueFactory.bootstrapping(Collections.Tokensingleton(keyTokens.get(7;
+PendingRangeCalculatorService.instance.blockUntilFinished();
 
 // don't require test update every time a new keyspace is added to 
test/conf/cassandra.yaml
 MapString, AbstractReplicationStrategy tableStrategyMap = new 
HashMapString, AbstractReplicationStrategy();



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

2013-12-04 Thread marcuse
Merge branch 'cassandra-1.2' into cassandra-2.0


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

Branch: refs/heads/cassandra-2.0
Commit: 32dbe58254115b8a92a97e95566bb4374bb9c051
Parents: 1334f94 d8c4e89
Author: Marcus Eriksson marc...@spotify.com
Authored: Wed Dec 4 20:17:38 2013 +0100
Committer: Marcus Eriksson marc...@spotify.com
Committed: Wed Dec 4 20:17:38 2013 +0100

--
 test/unit/org/apache/cassandra/service/MoveTest.java | 2 ++
 1 file changed, 2 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/32dbe582/test/unit/org/apache/cassandra/service/MoveTest.java
--
diff --cc test/unit/org/apache/cassandra/service/MoveTest.java
index f239671,07a9590..6ecd500
--- a/test/unit/org/apache/cassandra/service/MoveTest.java
+++ b/test/unit/org/apache/cassandra/service/MoveTest.java
@@@ -197,12 -198,13 +198,13 @@@ public class MoveTes
  ss.onChange(boot2,
  ApplicationState.STATUS,
  
valueFactory.bootstrapping(Collections.Tokensingleton(keyTokens.get(7;
+ PendingRangeCalculatorService.instance.blockUntilFinished();
  
  // don't require test update every time a new keyspace is added to 
test/conf/cassandra.yaml
 -MapString, AbstractReplicationStrategy tableStrategyMap = new 
HashMapString, AbstractReplicationStrategy();
 +MapString, AbstractReplicationStrategy keyspaceStrategyMap = new 
HashMapString, AbstractReplicationStrategy();
  for (int i = 1; i = 4; i++)
  {
 -tableStrategyMap.put(Keyspace + i, getStrategy(Keyspace + i, 
tmd));
 +keyspaceStrategyMap.put(Keyspace + i, getStrategy(Keyspace + 
i, tmd));
  }
  
 /**



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

2013-12-04 Thread marcuse
Merge branch 'cassandra-1.2' into cassandra-2.0


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

Branch: refs/heads/trunk
Commit: 32dbe58254115b8a92a97e95566bb4374bb9c051
Parents: 1334f94 d8c4e89
Author: Marcus Eriksson marc...@spotify.com
Authored: Wed Dec 4 20:17:38 2013 +0100
Committer: Marcus Eriksson marc...@spotify.com
Committed: Wed Dec 4 20:17:38 2013 +0100

--
 test/unit/org/apache/cassandra/service/MoveTest.java | 2 ++
 1 file changed, 2 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/32dbe582/test/unit/org/apache/cassandra/service/MoveTest.java
--
diff --cc test/unit/org/apache/cassandra/service/MoveTest.java
index f239671,07a9590..6ecd500
--- a/test/unit/org/apache/cassandra/service/MoveTest.java
+++ b/test/unit/org/apache/cassandra/service/MoveTest.java
@@@ -197,12 -198,13 +198,13 @@@ public class MoveTes
  ss.onChange(boot2,
  ApplicationState.STATUS,
  
valueFactory.bootstrapping(Collections.Tokensingleton(keyTokens.get(7;
+ PendingRangeCalculatorService.instance.blockUntilFinished();
  
  // don't require test update every time a new keyspace is added to 
test/conf/cassandra.yaml
 -MapString, AbstractReplicationStrategy tableStrategyMap = new 
HashMapString, AbstractReplicationStrategy();
 +MapString, AbstractReplicationStrategy keyspaceStrategyMap = new 
HashMapString, AbstractReplicationStrategy();
  for (int i = 1; i = 4; i++)
  {
 -tableStrategyMap.put(Keyspace + i, getStrategy(Keyspace + i, 
tmd));
 +keyspaceStrategyMap.put(Keyspace + i, getStrategy(Keyspace + 
i, tmd));
  }
  
 /**



[2/2] git commit: Merge branch 'cassandra-2.1' into trunk

2014-02-24 Thread marcuse
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: e4148684725151aef4c0489c0c809f3983dabe37
Parents: 3b45d00 f0c9bd1
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Feb 24 10:22:56 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Feb 24 10:22:56 2014 +0100

--
 CHANGES.txt |  2 +-
 .../cassandra/db/compaction/Scrubber.java   | 11 +++--
 .../cassandra/io/sstable/SSTableWriter.java | 14 +-
 tools/bin/sstablerepairedset| 46 
 4 files changed, 68 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e4148684/CHANGES.txt
--



[1/2] git commit: Scrub should not always clear out repaired status.

2014-02-24 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/trunk 3b45d004c - e41486847


Scrub should not always clear out repaired status.

Patch by marcuse, reviewed by jbellis for CASSANDRA-5351


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

Branch: refs/heads/trunk
Commit: f0c9bd1c9da8dcacf2e9268bb1855b92f5e05e61
Parents: e30f111
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Feb 24 10:16:25 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Feb 24 10:21:33 2014 +0100

--
 CHANGES.txt |  2 +-
 .../cassandra/db/compaction/Scrubber.java   | 11 +++--
 .../cassandra/io/sstable/SSTableWriter.java | 14 +-
 tools/bin/sstablerepairedset| 46 
 4 files changed, 68 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f0c9bd1c/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index be2925a..cd58a74 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,7 +4,7 @@
  * Allow nodetool to use a file or prompt for password (CASSANDRA-6660)
  * Fix AIOOBE when concurrently accessing ABSC (CASSANDRA-6742)
  * Fix assertion error in ALTER TYPE RENAME (CASSANDRA-6705)
-
+ * Scrub should not always clear out repaired status (CASSANDRA-5351)
 
 2.1.0-beta1
  * Add flush directory distinct from compaction directories (CASSANDRA-6357)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f0c9bd1c/src/java/org/apache/cassandra/db/compaction/Scrubber.java
--
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java 
b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index 3a71136..0a1e8c4 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -114,7 +114,7 @@ public class Scrubber implements Closeable
 }
 
 // TODO errors when creating the writer may leave empty temp files.
-writer = CompactionManager.createWriter(cfs, destination, 
expectedBloomFilterSize, ActiveRepairService.UNREPAIRED_SSTABLE, sstable);
+writer = CompactionManager.createWriter(cfs, destination, 
expectedBloomFilterSize, sstable.getSSTableMetadata().repairedAt, sstable);
 
 DecoratedKey prevKey = null;
 
@@ -257,7 +257,10 @@ public class Scrubber implements Closeable
 }
 
 if (writer.getFilePointer()  0)
-newSstable = writer.closeAndOpenReader(sstable.maxDataAge);
+{
+long repairedAt = badRows  0 ? 
ActiveRepairService.UNREPAIRED_SSTABLE : 
sstable.getSSTableMetadata().repairedAt;
+newSstable = writer.closeAndOpenReader(sstable.maxDataAge, 
repairedAt);
+}
 }
 catch (Throwable t)
 {
@@ -272,7 +275,9 @@ public class Scrubber implements Closeable
 
 if (!outOfOrderRows.isEmpty())
 {
-SSTableWriter inOrderWriter = CompactionManager.createWriter(cfs, 
destination, expectedBloomFilterSize, ActiveRepairService.UNREPAIRED_SSTABLE, 
sstable);
+// out of order rows, but no bad rows found - we can keep our 
repairedAt time
+long repairedAt = badRows  0 ? 
ActiveRepairService.UNREPAIRED_SSTABLE : 
sstable.getSSTableMetadata().repairedAt;
+SSTableWriter inOrderWriter = CompactionManager.createWriter(cfs, 
destination, expectedBloomFilterSize, repairedAt, sstable);
 for (Row row : outOfOrderRows)
 inOrderWriter.append(row.key, row.cf);
 newInOrderSstable = 
inOrderWriter.closeAndOpenReader(sstable.maxDataAge);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f0c9bd1c/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
--
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java 
b/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
index ab1fe49..2d1858f 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
@@ -330,7 +330,12 @@ public class SSTableWriter extends SSTable
 
 public SSTableReader closeAndOpenReader(long maxDataAge)
 {
-PairDescriptor, StatsMetadata p = close();
+return closeAndOpenReader(maxDataAge, this.repairedAt);
+}
+
+public SSTableReader closeAndOpenReader(long maxDataAge, long repairedAt

git commit: Scrub should not always clear out repaired status.

2014-02-24 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 e30f11143 - f0c9bd1c9


Scrub should not always clear out repaired status.

Patch by marcuse, reviewed by jbellis for CASSANDRA-5351


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

Branch: refs/heads/cassandra-2.1
Commit: f0c9bd1c9da8dcacf2e9268bb1855b92f5e05e61
Parents: e30f111
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Feb 24 10:16:25 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Feb 24 10:21:33 2014 +0100

--
 CHANGES.txt |  2 +-
 .../cassandra/db/compaction/Scrubber.java   | 11 +++--
 .../cassandra/io/sstable/SSTableWriter.java | 14 +-
 tools/bin/sstablerepairedset| 46 
 4 files changed, 68 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f0c9bd1c/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index be2925a..cd58a74 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,7 +4,7 @@
  * Allow nodetool to use a file or prompt for password (CASSANDRA-6660)
  * Fix AIOOBE when concurrently accessing ABSC (CASSANDRA-6742)
  * Fix assertion error in ALTER TYPE RENAME (CASSANDRA-6705)
-
+ * Scrub should not always clear out repaired status (CASSANDRA-5351)
 
 2.1.0-beta1
  * Add flush directory distinct from compaction directories (CASSANDRA-6357)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f0c9bd1c/src/java/org/apache/cassandra/db/compaction/Scrubber.java
--
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java 
b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index 3a71136..0a1e8c4 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -114,7 +114,7 @@ public class Scrubber implements Closeable
 }
 
 // TODO errors when creating the writer may leave empty temp files.
-writer = CompactionManager.createWriter(cfs, destination, 
expectedBloomFilterSize, ActiveRepairService.UNREPAIRED_SSTABLE, sstable);
+writer = CompactionManager.createWriter(cfs, destination, 
expectedBloomFilterSize, sstable.getSSTableMetadata().repairedAt, sstable);
 
 DecoratedKey prevKey = null;
 
@@ -257,7 +257,10 @@ public class Scrubber implements Closeable
 }
 
 if (writer.getFilePointer()  0)
-newSstable = writer.closeAndOpenReader(sstable.maxDataAge);
+{
+long repairedAt = badRows  0 ? 
ActiveRepairService.UNREPAIRED_SSTABLE : 
sstable.getSSTableMetadata().repairedAt;
+newSstable = writer.closeAndOpenReader(sstable.maxDataAge, 
repairedAt);
+}
 }
 catch (Throwable t)
 {
@@ -272,7 +275,9 @@ public class Scrubber implements Closeable
 
 if (!outOfOrderRows.isEmpty())
 {
-SSTableWriter inOrderWriter = CompactionManager.createWriter(cfs, 
destination, expectedBloomFilterSize, ActiveRepairService.UNREPAIRED_SSTABLE, 
sstable);
+// out of order rows, but no bad rows found - we can keep our 
repairedAt time
+long repairedAt = badRows  0 ? 
ActiveRepairService.UNREPAIRED_SSTABLE : 
sstable.getSSTableMetadata().repairedAt;
+SSTableWriter inOrderWriter = CompactionManager.createWriter(cfs, 
destination, expectedBloomFilterSize, repairedAt, sstable);
 for (Row row : outOfOrderRows)
 inOrderWriter.append(row.key, row.cf);
 newInOrderSstable = 
inOrderWriter.closeAndOpenReader(sstable.maxDataAge);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f0c9bd1c/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
--
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java 
b/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
index ab1fe49..2d1858f 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
@@ -330,7 +330,12 @@ public class SSTableWriter extends SSTable
 
 public SSTableReader closeAndOpenReader(long maxDataAge)
 {
-PairDescriptor, StatsMetadata p = close();
+return closeAndOpenReader(maxDataAge, this.repairedAt);
+}
+
+public SSTableReader closeAndOpenReader(long maxDataAge, long repairedAt

git commit: Fix resetAndTruncate:ing CompressionMetadata

2014-03-03 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.0 b3a9a4434 - 41d8a5f48


Fix resetAndTruncate:ing CompressionMetadata

Patch by kvaster, reviewed by marcuse for CASSANDRA-6791


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

Branch: refs/heads/cassandra-2.0
Commit: 41d8a5f4861c37ff8e22344418e9277236672c1f
Parents: b3a9a44
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 11:35:03 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 14:21:00 2014 +0100

--
 CHANGES.txt |  1 +
 .../io/compress/CompressedSequentialWriter.java |  2 +-
 .../CompressedRandomAccessReaderTest.java   | 42 
 3 files changed, 44 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/41d8a5f4/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 3e73f91..6de11c5 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -28,6 +28,7 @@
  * Disallow post-query re-ordering when paging (CASSANDRA-6722)
  * Fix potential paging bug with deleted columns (CASSANDRA-6748)
  * Fix NPE on BulkLoader caused by losing StreamEvent (CASSANDRA-6636)
+ * Fix truncating compression metadata (CASSANDRA-6791)
 Merged from 1.2:
  * Add CMSClassUnloadingEnabled JVM option (CASSANDRA-6541)
  * Catch memtable flush exceptions during shutdown (CASSANDRA-6735)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41d8a5f4/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
--
diff --git 
a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java 
b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index 54b990f..eef5b17 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -231,7 +231,7 @@ public class CompressedSequentialWriter extends 
SequentialWriter
 
 // truncate data and index file
 truncate(chunkOffset);
-metadataWriter.resetAndTruncate(realMark.nextChunkIndex);
+metadataWriter.resetAndTruncate(realMark.nextChunkIndex - 1);
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41d8a5f4/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
--
diff --git 
a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
 
b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
index ee32a0e..3c9dfe5 100644
--- 
a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
+++ 
b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
@@ -19,11 +19,13 @@
 package org.apache.cassandra.io.compress;
 
 import java.io.*;
+import java.util.Collections;
 import java.util.Random;
 
 import org.junit.Test;
 
 import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.SSTableMetadata;
 import org.apache.cassandra.io.util.*;
@@ -48,6 +50,46 @@ public class CompressedRandomAccessReaderTest
 testResetAndTruncate(File.createTempFile(compressed, 1), true, 10);
 testResetAndTruncate(File.createTempFile(compressed, 2), true, 
CompressionParameters.DEFAULT_CHUNK_LENGTH);
 }
+@Test
+public void test6791() throws IOException, ConfigurationException
+{
+File f = File.createTempFile(compressed6791_, 3);
+String filename = f.getAbsolutePath();
+try
+{
+
+SSTableMetadata.Collector sstableMetadataCollector = 
SSTableMetadata.createCollector(BytesType.instance).replayPosition(null);
+CompressedSequentialWriter writer = new 
CompressedSequentialWriter(f, filename + .metadata, false, new 
CompressionParameters(SnappyCompressor.instance, 32, Collections.String, 
StringemptyMap()), sstableMetadataCollector);
+
+for (int i = 0; i  20; i++)
+writer.write(x.getBytes());
+
+FileMark mark = writer.mark();
+// write enough garbage to create new chunks:
+for (int i = 0; i  40; ++i)
+writer.write(y.getBytes());
+
+writer.resetAndTruncate(mark);
+
+for (int i = 0; i  20; i++)
+writer.write

[3/3] git commit: Merge branch 'cassandra-2.1' into trunk

2014-03-03 Thread marcuse
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: e80564f398c6278f6a6a28943e06ec800d5e68f7
Parents: 7d8092b e4437bc
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 14:27:15 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 14:27:15 2014 +0100

--
 CHANGES.txt |  1 +
 .../io/compress/CompressedSequentialWriter.java |  2 +-
 .../CompressedRandomAccessReaderTest.java   | 42 
 3 files changed, 44 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e80564f3/CHANGES.txt
--



[1/3] git commit: Fix resetAndTruncate:ing CompressionMetadata

2014-03-03 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/trunk 7d8092b66 - e80564f39


Fix resetAndTruncate:ing CompressionMetadata

Patch by kvaster, reviewed by marcuse for CASSANDRA-6791


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

Branch: refs/heads/trunk
Commit: 41d8a5f4861c37ff8e22344418e9277236672c1f
Parents: b3a9a44
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 11:35:03 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 14:21:00 2014 +0100

--
 CHANGES.txt |  1 +
 .../io/compress/CompressedSequentialWriter.java |  2 +-
 .../CompressedRandomAccessReaderTest.java   | 42 
 3 files changed, 44 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/41d8a5f4/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 3e73f91..6de11c5 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -28,6 +28,7 @@
  * Disallow post-query re-ordering when paging (CASSANDRA-6722)
  * Fix potential paging bug with deleted columns (CASSANDRA-6748)
  * Fix NPE on BulkLoader caused by losing StreamEvent (CASSANDRA-6636)
+ * Fix truncating compression metadata (CASSANDRA-6791)
 Merged from 1.2:
  * Add CMSClassUnloadingEnabled JVM option (CASSANDRA-6541)
  * Catch memtable flush exceptions during shutdown (CASSANDRA-6735)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41d8a5f4/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
--
diff --git 
a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java 
b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index 54b990f..eef5b17 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -231,7 +231,7 @@ public class CompressedSequentialWriter extends 
SequentialWriter
 
 // truncate data and index file
 truncate(chunkOffset);
-metadataWriter.resetAndTruncate(realMark.nextChunkIndex);
+metadataWriter.resetAndTruncate(realMark.nextChunkIndex - 1);
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41d8a5f4/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
--
diff --git 
a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
 
b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
index ee32a0e..3c9dfe5 100644
--- 
a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
+++ 
b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
@@ -19,11 +19,13 @@
 package org.apache.cassandra.io.compress;
 
 import java.io.*;
+import java.util.Collections;
 import java.util.Random;
 
 import org.junit.Test;
 
 import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.SSTableMetadata;
 import org.apache.cassandra.io.util.*;
@@ -48,6 +50,46 @@ public class CompressedRandomAccessReaderTest
 testResetAndTruncate(File.createTempFile(compressed, 1), true, 10);
 testResetAndTruncate(File.createTempFile(compressed, 2), true, 
CompressionParameters.DEFAULT_CHUNK_LENGTH);
 }
+@Test
+public void test6791() throws IOException, ConfigurationException
+{
+File f = File.createTempFile(compressed6791_, 3);
+String filename = f.getAbsolutePath();
+try
+{
+
+SSTableMetadata.Collector sstableMetadataCollector = 
SSTableMetadata.createCollector(BytesType.instance).replayPosition(null);
+CompressedSequentialWriter writer = new 
CompressedSequentialWriter(f, filename + .metadata, false, new 
CompressionParameters(SnappyCompressor.instance, 32, Collections.String, 
StringemptyMap()), sstableMetadataCollector);
+
+for (int i = 0; i  20; i++)
+writer.write(x.getBytes());
+
+FileMark mark = writer.mark();
+// write enough garbage to create new chunks:
+for (int i = 0; i  40; ++i)
+writer.write(y.getBytes());
+
+writer.resetAndTruncate(mark);
+
+for (int i = 0; i  20; i++)
+writer.write(x.getBytes

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

2014-03-03 Thread marcuse
Merge branch 'cassandra-2.0' into cassandra-2.1

Conflicts:
CHANGES.txt


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

Branch: refs/heads/trunk
Commit: e4437bc1881aa58b6adbf2a9483e827e37ede8d5
Parents: dcca996 41d8a5f
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 14:26:40 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 14:26:40 2014 +0100

--
 CHANGES.txt |  1 +
 .../io/compress/CompressedSequentialWriter.java |  2 +-
 .../CompressedRandomAccessReaderTest.java   | 42 
 3 files changed, 44 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e4437bc1/CHANGES.txt
--
diff --cc CHANGES.txt
index 20ae300,6de11c5..516a0d1
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -36,47 -28,24 +36,48 @@@ Merged from 2.0
   * Disallow post-query re-ordering when paging (CASSANDRA-6722)
   * Fix potential paging bug with deleted columns (CASSANDRA-6748)
   * Fix NPE on BulkLoader caused by losing StreamEvent (CASSANDRA-6636)
+  * Fix truncating compression metadata (CASSANDRA-6791)
 -Merged from 1.2:
   * Add CMSClassUnloadingEnabled JVM option (CASSANDRA-6541)
   * Catch memtable flush exceptions during shutdown (CASSANDRA-6735)
 - * Fix broken streams when replacing with same IP (CASSANDRA-6622)
   * Fix upgradesstables NPE for non-CF-based indexes (CASSANDRA-6645)
 - * Fix partition and range deletes not triggering flush (CASSANDRA-6655)
 - * Fix mean cells and mean row size per sstable calculations (CASSANDRA-6667)
 - * Compact hints after partial replay to clean out tombstones (CASSANDRA-)
 - * Log USING TTL/TIMESTAMP in a counter update warning (CASSANDRA-6649)
 - * Don't exchange schema between nodes with different versions 
(CASSANDRA-6695)
 - * Use real node messaging versions for schema exchange decisions 
(CASSANDRA-6700)
 - * IN on the last clustering columns + ORDER BY DESC yield no results 
(CASSANDRA-6701)
 - * Fix SecondaryIndexManager#deleteFromIndexes() (CASSANDRA-6711)
 - * Fix snapshot repair not snapshotting coordinator itself (CASSANDRA-6713)
 - * Support negative timestamps for CQL3 dates in query string (CASSANDRA-6718)
 - * Avoid NPEs when receiving table changes for an unknown keyspace 
(CASSANDRA-5631)
 - * Fix bootstrapping when there is no schema (CASSANDRA-6685)
 +
 +
 +2.1.0-beta1
 + * Add flush directory distinct from compaction directories (CASSANDRA-6357)
 + * Require JNA by default (CASSANDRA-6575)
 + * add listsnapshots command to nodetool (CASSANDRA-5742)
 + * Introduce AtomicBTreeColumns (CASSANDRA-6271, 6692)
 + * Multithreaded commitlog (CASSANDRA-3578)
 + * allocate fixed index summary memory pool and resample cold index summaries 
 +   to use less memory (CASSANDRA-5519)
 + * Removed multithreaded compaction (CASSANDRA-6142)
 + * Parallelize fetching rows for low-cardinality indexes (CASSANDRA-1337)
 + * change logging from log4j to logback (CASSANDRA-5883)
 + * switch to LZ4 compression for internode communication (CASSANDRA-5887)
 + * Stop using Thrift-generated Index* classes internally (CASSANDRA-5971)
 + * Remove 1.2 network compatibility code (CASSANDRA-5960)
 + * Remove leveled json manifest migration code (CASSANDRA-5996)
 + * Remove CFDefinition (CASSANDRA-6253)
 + * Use AtomicIntegerFieldUpdater in RefCountedMemory (CASSANDRA-6278)
 + * User-defined types for CQL3 (CASSANDRA-5590)
 + * Use of o.a.c.metrics in nodetool (CASSANDRA-5871, 6406)
 + * Batch read from OTC's queue and cleanup (CASSANDRA-1632)
 + * Secondary index support for collections (CASSANDRA-4511, 6383)
 + * SSTable metadata(Stats.db) format change (CASSANDRA-6356)
 + * Push composites support in the storage engine
 +   (CASSANDRA-5417, CASSANDRA-6520)
 + * Add snapshot space used to cfstats (CASSANDRA-6231)
 + * Add cardinality estimator for key count estimation (CASSANDRA-5906)
 + * CF id is changed to be non-deterministic. Data dir/key cache are created
 +   uniquely for CF id (CASSANDRA-5202)
 + * New counters implementation (CASSANDRA-6504)
 + * Replace UnsortedColumns, EmptyColumns, TreeMapBackedSortedColumns with new
 +   ArrayBackedSortedColumns (CASSANDRA-6630, CASSANDRA-6662, CASSANDRA-6690)
 + * Add option to use row cache with a given amount of rows (CASSANDRA-5357)
 + * Avoid repairing already repaired data (CASSANDRA-5351)
 + * Reject counter updates with USING TTL/TIMESTAMP (CASSANDRA-6649)
 + * Replace index_interval with min/max_index_interval (CASSANDRA-6379)
 + * Lift limitation that order by columns must be selected for 

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

2014-03-03 Thread marcuse
Merge branch 'cassandra-2.0' into cassandra-2.1

Conflicts:
CHANGES.txt


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

Branch: refs/heads/cassandra-2.1
Commit: e4437bc1881aa58b6adbf2a9483e827e37ede8d5
Parents: dcca996 41d8a5f
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 14:26:40 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 14:26:40 2014 +0100

--
 CHANGES.txt |  1 +
 .../io/compress/CompressedSequentialWriter.java |  2 +-
 .../CompressedRandomAccessReaderTest.java   | 42 
 3 files changed, 44 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e4437bc1/CHANGES.txt
--
diff --cc CHANGES.txt
index 20ae300,6de11c5..516a0d1
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -36,47 -28,24 +36,48 @@@ Merged from 2.0
   * Disallow post-query re-ordering when paging (CASSANDRA-6722)
   * Fix potential paging bug with deleted columns (CASSANDRA-6748)
   * Fix NPE on BulkLoader caused by losing StreamEvent (CASSANDRA-6636)
+  * Fix truncating compression metadata (CASSANDRA-6791)
 -Merged from 1.2:
   * Add CMSClassUnloadingEnabled JVM option (CASSANDRA-6541)
   * Catch memtable flush exceptions during shutdown (CASSANDRA-6735)
 - * Fix broken streams when replacing with same IP (CASSANDRA-6622)
   * Fix upgradesstables NPE for non-CF-based indexes (CASSANDRA-6645)
 - * Fix partition and range deletes not triggering flush (CASSANDRA-6655)
 - * Fix mean cells and mean row size per sstable calculations (CASSANDRA-6667)
 - * Compact hints after partial replay to clean out tombstones (CASSANDRA-)
 - * Log USING TTL/TIMESTAMP in a counter update warning (CASSANDRA-6649)
 - * Don't exchange schema between nodes with different versions 
(CASSANDRA-6695)
 - * Use real node messaging versions for schema exchange decisions 
(CASSANDRA-6700)
 - * IN on the last clustering columns + ORDER BY DESC yield no results 
(CASSANDRA-6701)
 - * Fix SecondaryIndexManager#deleteFromIndexes() (CASSANDRA-6711)
 - * Fix snapshot repair not snapshotting coordinator itself (CASSANDRA-6713)
 - * Support negative timestamps for CQL3 dates in query string (CASSANDRA-6718)
 - * Avoid NPEs when receiving table changes for an unknown keyspace 
(CASSANDRA-5631)
 - * Fix bootstrapping when there is no schema (CASSANDRA-6685)
 +
 +
 +2.1.0-beta1
 + * Add flush directory distinct from compaction directories (CASSANDRA-6357)
 + * Require JNA by default (CASSANDRA-6575)
 + * add listsnapshots command to nodetool (CASSANDRA-5742)
 + * Introduce AtomicBTreeColumns (CASSANDRA-6271, 6692)
 + * Multithreaded commitlog (CASSANDRA-3578)
 + * allocate fixed index summary memory pool and resample cold index summaries 
 +   to use less memory (CASSANDRA-5519)
 + * Removed multithreaded compaction (CASSANDRA-6142)
 + * Parallelize fetching rows for low-cardinality indexes (CASSANDRA-1337)
 + * change logging from log4j to logback (CASSANDRA-5883)
 + * switch to LZ4 compression for internode communication (CASSANDRA-5887)
 + * Stop using Thrift-generated Index* classes internally (CASSANDRA-5971)
 + * Remove 1.2 network compatibility code (CASSANDRA-5960)
 + * Remove leveled json manifest migration code (CASSANDRA-5996)
 + * Remove CFDefinition (CASSANDRA-6253)
 + * Use AtomicIntegerFieldUpdater in RefCountedMemory (CASSANDRA-6278)
 + * User-defined types for CQL3 (CASSANDRA-5590)
 + * Use of o.a.c.metrics in nodetool (CASSANDRA-5871, 6406)
 + * Batch read from OTC's queue and cleanup (CASSANDRA-1632)
 + * Secondary index support for collections (CASSANDRA-4511, 6383)
 + * SSTable metadata(Stats.db) format change (CASSANDRA-6356)
 + * Push composites support in the storage engine
 +   (CASSANDRA-5417, CASSANDRA-6520)
 + * Add snapshot space used to cfstats (CASSANDRA-6231)
 + * Add cardinality estimator for key count estimation (CASSANDRA-5906)
 + * CF id is changed to be non-deterministic. Data dir/key cache are created
 +   uniquely for CF id (CASSANDRA-5202)
 + * New counters implementation (CASSANDRA-6504)
 + * Replace UnsortedColumns, EmptyColumns, TreeMapBackedSortedColumns with new
 +   ArrayBackedSortedColumns (CASSANDRA-6630, CASSANDRA-6662, CASSANDRA-6690)
 + * Add option to use row cache with a given amount of rows (CASSANDRA-5357)
 + * Avoid repairing already repaired data (CASSANDRA-5351)
 + * Reject counter updates with USING TTL/TIMESTAMP (CASSANDRA-6649)
 + * Replace index_interval with min/max_index_interval (CASSANDRA-6379)
 + * Lift limitation that order by columns must be 

[1/2] git commit: Fix resetAndTruncate:ing CompressionMetadata

2014-03-03 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 dcca99684 - e4437bc18


Fix resetAndTruncate:ing CompressionMetadata

Patch by kvaster, reviewed by marcuse for CASSANDRA-6791


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

Branch: refs/heads/cassandra-2.1
Commit: 41d8a5f4861c37ff8e22344418e9277236672c1f
Parents: b3a9a44
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 11:35:03 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 14:21:00 2014 +0100

--
 CHANGES.txt |  1 +
 .../io/compress/CompressedSequentialWriter.java |  2 +-
 .../CompressedRandomAccessReaderTest.java   | 42 
 3 files changed, 44 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/41d8a5f4/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 3e73f91..6de11c5 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -28,6 +28,7 @@
  * Disallow post-query re-ordering when paging (CASSANDRA-6722)
  * Fix potential paging bug with deleted columns (CASSANDRA-6748)
  * Fix NPE on BulkLoader caused by losing StreamEvent (CASSANDRA-6636)
+ * Fix truncating compression metadata (CASSANDRA-6791)
 Merged from 1.2:
  * Add CMSClassUnloadingEnabled JVM option (CASSANDRA-6541)
  * Catch memtable flush exceptions during shutdown (CASSANDRA-6735)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41d8a5f4/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
--
diff --git 
a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java 
b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index 54b990f..eef5b17 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -231,7 +231,7 @@ public class CompressedSequentialWriter extends 
SequentialWriter
 
 // truncate data and index file
 truncate(chunkOffset);
-metadataWriter.resetAndTruncate(realMark.nextChunkIndex);
+metadataWriter.resetAndTruncate(realMark.nextChunkIndex - 1);
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41d8a5f4/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
--
diff --git 
a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
 
b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
index ee32a0e..3c9dfe5 100644
--- 
a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
+++ 
b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
@@ -19,11 +19,13 @@
 package org.apache.cassandra.io.compress;
 
 import java.io.*;
+import java.util.Collections;
 import java.util.Random;
 
 import org.junit.Test;
 
 import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.SSTableMetadata;
 import org.apache.cassandra.io.util.*;
@@ -48,6 +50,46 @@ public class CompressedRandomAccessReaderTest
 testResetAndTruncate(File.createTempFile(compressed, 1), true, 10);
 testResetAndTruncate(File.createTempFile(compressed, 2), true, 
CompressionParameters.DEFAULT_CHUNK_LENGTH);
 }
+@Test
+public void test6791() throws IOException, ConfigurationException
+{
+File f = File.createTempFile(compressed6791_, 3);
+String filename = f.getAbsolutePath();
+try
+{
+
+SSTableMetadata.Collector sstableMetadataCollector = 
SSTableMetadata.createCollector(BytesType.instance).replayPosition(null);
+CompressedSequentialWriter writer = new 
CompressedSequentialWriter(f, filename + .metadata, false, new 
CompressionParameters(SnappyCompressor.instance, 32, Collections.String, 
StringemptyMap()), sstableMetadataCollector);
+
+for (int i = 0; i  20; i++)
+writer.write(x.getBytes());
+
+FileMark mark = writer.mark();
+// write enough garbage to create new chunks:
+for (int i = 0; i  40; ++i)
+writer.write(y.getBytes());
+
+writer.resetAndTruncate(mark);
+
+for (int i = 0; i  20; i++)
+writer.write

git commit: Fix merge

2014-03-03 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 e4437bc18 - 043953127


Fix merge


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

Branch: refs/heads/cassandra-2.1
Commit: 04395312745805ed7e4fd201b88bcd3d46b3a601
Parents: e4437bc
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 14:39:40 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 14:39:40 2014 +0100

--
 .../cassandra/io/compress/CompressedRandomAccessReaderTest.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/04395312/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
--
diff --git 
a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
 
b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
index c9ddfea..f8fcf76 100644
--- 
a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
+++ 
b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
@@ -29,6 +29,7 @@ import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
 import org.apache.cassandra.io.util.*;
 
 import static org.junit.Assert.assertEquals;
@@ -59,7 +60,7 @@ public class CompressedRandomAccessReaderTest
 try
 {
 
-SSTableMetadata.Collector sstableMetadataCollector = 
SSTableMetadata.createCollector(BytesType.instance).replayPosition(null);
+MetadataCollector sstableMetadataCollector = new 
MetadataCollector(new SimpleDenseCellNameType(BytesType.instance));
 CompressedSequentialWriter writer = new 
CompressedSequentialWriter(f, filename + .metadata, false, new 
CompressionParameters(SnappyCompressor.instance, 32, Collections.String, 
StringemptyMap()), sstableMetadataCollector);
 
 for (int i = 0; i  20; i++)



[2/2] git commit: Merge branch 'cassandra-2.1' into trunk

2014-03-03 Thread marcuse
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 2c8100af210990bf1f15cc6691066787b88e5f25
Parents: e80564f 0439531
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 14:39:45 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 14:39:45 2014 +0100

--
 .../cassandra/io/compress/CompressedRandomAccessReaderTest.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--




[1/2] git commit: Fix merge

2014-03-03 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/trunk e80564f39 - 2c8100af2


Fix merge


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

Branch: refs/heads/trunk
Commit: 04395312745805ed7e4fd201b88bcd3d46b3a601
Parents: e4437bc
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 14:39:40 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 14:39:40 2014 +0100

--
 .../cassandra/io/compress/CompressedRandomAccessReaderTest.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/04395312/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
--
diff --git 
a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
 
b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
index c9ddfea..f8fcf76 100644
--- 
a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
+++ 
b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
@@ -29,6 +29,7 @@ import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
 import org.apache.cassandra.io.util.*;
 
 import static org.junit.Assert.assertEquals;
@@ -59,7 +60,7 @@ public class CompressedRandomAccessReaderTest
 try
 {
 
-SSTableMetadata.Collector sstableMetadataCollector = 
SSTableMetadata.createCollector(BytesType.instance).replayPosition(null);
+MetadataCollector sstableMetadataCollector = new 
MetadataCollector(new SimpleDenseCellNameType(BytesType.instance));
 CompressedSequentialWriter writer = new 
CompressedSequentialWriter(f, filename + .metadata, false, new 
CompressionParameters(SnappyCompressor.instance, 32, Collections.String, 
StringemptyMap()), sstableMetadataCollector);
 
 for (int i = 0; i  20; i++)



git commit: Fix resetAndTruncate:ing CompressionMetadata

2014-03-03 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-1.2 f08ae394f - 64098f7d6


Fix resetAndTruncate:ing CompressionMetadata

Patch by kvaster, reviewed by marcuse for CASSANDRA-6791


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

Branch: refs/heads/cassandra-1.2
Commit: 64098f7d6f0b122448693a3c6da16af54c99013b
Parents: f08ae39
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 15:03:34 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 15:03:34 2014 +0100

--
 CHANGES.txt |  2 +-
 .../io/compress/CompressedSequentialWriter.java |  2 +-
 .../CompressedRandomAccessReaderTest.java   | 43 
 3 files changed, 45 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/64098f7d/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 780b528..b3c0a35 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -19,7 +19,7 @@
  * Support negative timestamps for CQL3 dates in query string (CASSANDRA-6718)
  * Avoid NPEs when receiving table changes for an unknown keyspace 
(CASSANDRA-5631)
  * Fix bootstrapping when there is no schema (CASSANDRA-6685)
-
+ * Fix truncating compression metadata (CASSANDRA-6791)
 
 1.2.15
  * Move handling of migration event source to solve bootstrap race 
(CASSANDRA-6648)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/64098f7d/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
--
diff --git 
a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java 
b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index 00eb5a7..da55e83 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -231,7 +231,7 @@ public class CompressedSequentialWriter extends 
SequentialWriter
 
 // truncate data and index file
 truncate(chunkOffset);
-metadataWriter.resetAndTruncate(realMark.nextChunkIndex);
+metadataWriter.resetAndTruncate(realMark.nextChunkIndex - 1);
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/64098f7d/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
--
diff --git 
a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
 
b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
index 830c3e1..678a650 100644
--- 
a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
+++ 
b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
@@ -19,10 +19,12 @@
 package org.apache.cassandra.io.compress;
 
 import java.io.*;
+import java.util.Collections;
 import java.util.Random;
 
 import org.junit.Test;
 
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.SSTableMetadata;
 import org.apache.cassandra.io.util.*;
@@ -95,6 +97,47 @@ public class CompressedRandomAccessReaderTest
 }
 
 @Test
+public void test6791() throws IOException, ConfigurationException
+{
+File f = File.createTempFile(compressed6791_, 3);
+String filename = f.getAbsolutePath();
+try
+{
+
+SSTableMetadata.Collector sstableMetadataCollector = 
SSTableMetadata.createCollector().replayPosition(null);
+CompressedSequentialWriter writer = new 
CompressedSequentialWriter(f, filename + .metadata, false, new 
CompressionParameters(SnappyCompressor.instance, 32, Collections.String, 
StringemptyMap()), sstableMetadataCollector);
+
+for (int i = 0; i  20; i++)
+writer.write(x.getBytes());
+
+FileMark mark = writer.mark();
+// write enough garbage to create new chunks:
+for (int i = 0; i  40; i++)
+writer.write(y.getBytes());
+
+writer.resetAndTruncate(mark);
+
+for (int i = 0; i  20; i++)
+writer.write(x.getBytes());
+writer.close();
+
+CompressedRandomAccessReader reader = 
CompressedRandomAccessReader.open(filename, new CompressionMetadata(filename + 
.metadata, f.length()), false);
+String res = reader.readLine();
+assertEquals(res

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

2014-03-03 Thread marcuse
Merge branch 'cassandra-1.2' into cassandra-2.0


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

Branch: refs/heads/cassandra-2.0
Commit: ef20671c2c99ab40362785f4718c9e2c13edbda2
Parents: 41d8a5f 64098f7
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 15:04:00 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 15:04:00 2014 +0100

--

--




[4/4] git commit: Merge branch 'cassandra-2.1' into trunk

2014-03-03 Thread marcuse
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: db6b563f143dc5f2037244854527cafc8594e730
Parents: 2c8100a 56631e1
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 15:04:17 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 15:04:17 2014 +0100

--

--




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

2014-03-03 Thread marcuse
Merge branch 'cassandra-1.2' into cassandra-2.0


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

Branch: refs/heads/trunk
Commit: ef20671c2c99ab40362785f4718c9e2c13edbda2
Parents: 41d8a5f 64098f7
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 15:04:00 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 15:04:00 2014 +0100

--

--




[3/3] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

2014-03-03 Thread marcuse
Merge branch 'cassandra-2.0' into cassandra-2.1


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

Branch: refs/heads/cassandra-2.1
Commit: 56631e139092d81d55ea8c7c7ddccb3ff16eef62
Parents: 0439531 ef20671
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 15:04:10 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 15:04:10 2014 +0100

--

--




[3/4] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

2014-03-03 Thread marcuse
Merge branch 'cassandra-2.0' into cassandra-2.1


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

Branch: refs/heads/trunk
Commit: 56631e139092d81d55ea8c7c7ddccb3ff16eef62
Parents: 0439531 ef20671
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 15:04:10 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 15:04:10 2014 +0100

--

--




[1/2] git commit: Fix resetAndTruncate:ing CompressionMetadata

2014-03-03 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.0 41d8a5f48 - ef20671c2


Fix resetAndTruncate:ing CompressionMetadata

Patch by kvaster, reviewed by marcuse for CASSANDRA-6791


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

Branch: refs/heads/cassandra-2.0
Commit: 64098f7d6f0b122448693a3c6da16af54c99013b
Parents: f08ae39
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 15:03:34 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 15:03:34 2014 +0100

--
 CHANGES.txt |  2 +-
 .../io/compress/CompressedSequentialWriter.java |  2 +-
 .../CompressedRandomAccessReaderTest.java   | 43 
 3 files changed, 45 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/64098f7d/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 780b528..b3c0a35 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -19,7 +19,7 @@
  * Support negative timestamps for CQL3 dates in query string (CASSANDRA-6718)
  * Avoid NPEs when receiving table changes for an unknown keyspace 
(CASSANDRA-5631)
  * Fix bootstrapping when there is no schema (CASSANDRA-6685)
-
+ * Fix truncating compression metadata (CASSANDRA-6791)
 
 1.2.15
  * Move handling of migration event source to solve bootstrap race 
(CASSANDRA-6648)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/64098f7d/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
--
diff --git 
a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java 
b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index 00eb5a7..da55e83 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -231,7 +231,7 @@ public class CompressedSequentialWriter extends 
SequentialWriter
 
 // truncate data and index file
 truncate(chunkOffset);
-metadataWriter.resetAndTruncate(realMark.nextChunkIndex);
+metadataWriter.resetAndTruncate(realMark.nextChunkIndex - 1);
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/64098f7d/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
--
diff --git 
a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
 
b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
index 830c3e1..678a650 100644
--- 
a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
+++ 
b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
@@ -19,10 +19,12 @@
 package org.apache.cassandra.io.compress;
 
 import java.io.*;
+import java.util.Collections;
 import java.util.Random;
 
 import org.junit.Test;
 
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.SSTableMetadata;
 import org.apache.cassandra.io.util.*;
@@ -95,6 +97,47 @@ public class CompressedRandomAccessReaderTest
 }
 
 @Test
+public void test6791() throws IOException, ConfigurationException
+{
+File f = File.createTempFile(compressed6791_, 3);
+String filename = f.getAbsolutePath();
+try
+{
+
+SSTableMetadata.Collector sstableMetadataCollector = 
SSTableMetadata.createCollector().replayPosition(null);
+CompressedSequentialWriter writer = new 
CompressedSequentialWriter(f, filename + .metadata, false, new 
CompressionParameters(SnappyCompressor.instance, 32, Collections.String, 
StringemptyMap()), sstableMetadataCollector);
+
+for (int i = 0; i  20; i++)
+writer.write(x.getBytes());
+
+FileMark mark = writer.mark();
+// write enough garbage to create new chunks:
+for (int i = 0; i  40; i++)
+writer.write(y.getBytes());
+
+writer.resetAndTruncate(mark);
+
+for (int i = 0; i  20; i++)
+writer.write(x.getBytes());
+writer.close();
+
+CompressedRandomAccessReader reader = 
CompressedRandomAccessReader.open(filename, new CompressionMetadata(filename + 
.metadata, f.length()), false);
+String res = reader.readLine();
+assertEquals(res

[1/3] git commit: Fix resetAndTruncate:ing CompressionMetadata

2014-03-03 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 043953127 - 56631e139


Fix resetAndTruncate:ing CompressionMetadata

Patch by kvaster, reviewed by marcuse for CASSANDRA-6791


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

Branch: refs/heads/cassandra-2.1
Commit: 64098f7d6f0b122448693a3c6da16af54c99013b
Parents: f08ae39
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 15:03:34 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 15:03:34 2014 +0100

--
 CHANGES.txt |  2 +-
 .../io/compress/CompressedSequentialWriter.java |  2 +-
 .../CompressedRandomAccessReaderTest.java   | 43 
 3 files changed, 45 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/64098f7d/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 780b528..b3c0a35 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -19,7 +19,7 @@
  * Support negative timestamps for CQL3 dates in query string (CASSANDRA-6718)
  * Avoid NPEs when receiving table changes for an unknown keyspace 
(CASSANDRA-5631)
  * Fix bootstrapping when there is no schema (CASSANDRA-6685)
-
+ * Fix truncating compression metadata (CASSANDRA-6791)
 
 1.2.15
  * Move handling of migration event source to solve bootstrap race 
(CASSANDRA-6648)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/64098f7d/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
--
diff --git 
a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java 
b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index 00eb5a7..da55e83 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -231,7 +231,7 @@ public class CompressedSequentialWriter extends 
SequentialWriter
 
 // truncate data and index file
 truncate(chunkOffset);
-metadataWriter.resetAndTruncate(realMark.nextChunkIndex);
+metadataWriter.resetAndTruncate(realMark.nextChunkIndex - 1);
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/64098f7d/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
--
diff --git 
a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
 
b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
index 830c3e1..678a650 100644
--- 
a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
+++ 
b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
@@ -19,10 +19,12 @@
 package org.apache.cassandra.io.compress;
 
 import java.io.*;
+import java.util.Collections;
 import java.util.Random;
 
 import org.junit.Test;
 
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.SSTableMetadata;
 import org.apache.cassandra.io.util.*;
@@ -95,6 +97,47 @@ public class CompressedRandomAccessReaderTest
 }
 
 @Test
+public void test6791() throws IOException, ConfigurationException
+{
+File f = File.createTempFile(compressed6791_, 3);
+String filename = f.getAbsolutePath();
+try
+{
+
+SSTableMetadata.Collector sstableMetadataCollector = 
SSTableMetadata.createCollector().replayPosition(null);
+CompressedSequentialWriter writer = new 
CompressedSequentialWriter(f, filename + .metadata, false, new 
CompressionParameters(SnappyCompressor.instance, 32, Collections.String, 
StringemptyMap()), sstableMetadataCollector);
+
+for (int i = 0; i  20; i++)
+writer.write(x.getBytes());
+
+FileMark mark = writer.mark();
+// write enough garbage to create new chunks:
+for (int i = 0; i  40; i++)
+writer.write(y.getBytes());
+
+writer.resetAndTruncate(mark);
+
+for (int i = 0; i  20; i++)
+writer.write(x.getBytes());
+writer.close();
+
+CompressedRandomAccessReader reader = 
CompressedRandomAccessReader.open(filename, new CompressionMetadata(filename + 
.metadata, f.length()), false);
+String res = reader.readLine();
+assertEquals(res

[1/4] git commit: Fix resetAndTruncate:ing CompressionMetadata

2014-03-03 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/trunk 2c8100af2 - db6b563f1


Fix resetAndTruncate:ing CompressionMetadata

Patch by kvaster, reviewed by marcuse for CASSANDRA-6791


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

Branch: refs/heads/trunk
Commit: 64098f7d6f0b122448693a3c6da16af54c99013b
Parents: f08ae39
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 15:03:34 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 15:03:34 2014 +0100

--
 CHANGES.txt |  2 +-
 .../io/compress/CompressedSequentialWriter.java |  2 +-
 .../CompressedRandomAccessReaderTest.java   | 43 
 3 files changed, 45 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/64098f7d/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 780b528..b3c0a35 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -19,7 +19,7 @@
  * Support negative timestamps for CQL3 dates in query string (CASSANDRA-6718)
  * Avoid NPEs when receiving table changes for an unknown keyspace 
(CASSANDRA-5631)
  * Fix bootstrapping when there is no schema (CASSANDRA-6685)
-
+ * Fix truncating compression metadata (CASSANDRA-6791)
 
 1.2.15
  * Move handling of migration event source to solve bootstrap race 
(CASSANDRA-6648)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/64098f7d/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
--
diff --git 
a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java 
b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index 00eb5a7..da55e83 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -231,7 +231,7 @@ public class CompressedSequentialWriter extends 
SequentialWriter
 
 // truncate data and index file
 truncate(chunkOffset);
-metadataWriter.resetAndTruncate(realMark.nextChunkIndex);
+metadataWriter.resetAndTruncate(realMark.nextChunkIndex - 1);
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/64098f7d/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
--
diff --git 
a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
 
b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
index 830c3e1..678a650 100644
--- 
a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
+++ 
b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
@@ -19,10 +19,12 @@
 package org.apache.cassandra.io.compress;
 
 import java.io.*;
+import java.util.Collections;
 import java.util.Random;
 
 import org.junit.Test;
 
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.SSTableMetadata;
 import org.apache.cassandra.io.util.*;
@@ -95,6 +97,47 @@ public class CompressedRandomAccessReaderTest
 }
 
 @Test
+public void test6791() throws IOException, ConfigurationException
+{
+File f = File.createTempFile(compressed6791_, 3);
+String filename = f.getAbsolutePath();
+try
+{
+
+SSTableMetadata.Collector sstableMetadataCollector = 
SSTableMetadata.createCollector().replayPosition(null);
+CompressedSequentialWriter writer = new 
CompressedSequentialWriter(f, filename + .metadata, false, new 
CompressionParameters(SnappyCompressor.instance, 32, Collections.String, 
StringemptyMap()), sstableMetadataCollector);
+
+for (int i = 0; i  20; i++)
+writer.write(x.getBytes());
+
+FileMark mark = writer.mark();
+// write enough garbage to create new chunks:
+for (int i = 0; i  40; i++)
+writer.write(y.getBytes());
+
+writer.resetAndTruncate(mark);
+
+for (int i = 0; i  20; i++)
+writer.write(x.getBytes());
+writer.close();
+
+CompressedRandomAccessReader reader = 
CompressedRandomAccessReader.open(filename, new CompressionMetadata(filename + 
.metadata, f.length()), false);
+String res = reader.readLine();
+assertEquals(res

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

2014-03-03 Thread marcuse
Merge branch 'cassandra-1.2' into cassandra-2.0


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

Branch: refs/heads/cassandra-2.1
Commit: ef20671c2c99ab40362785f4718c9e2c13edbda2
Parents: 41d8a5f 64098f7
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 3 15:04:00 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 3 15:04:00 2014 +0100

--

--




git commit: Update syntax for changing caching options.

2014-03-12 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 ad5169d2f - da444a690


Update syntax for changing caching options.

Patch by marcuse, reviewed by slebresne for CASSANDRA-6745.


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

Branch: refs/heads/cassandra-2.1
Commit: da444a6903ddd8d02e7b251c8b38faf11d3a5c93
Parents: ad5169d
Author: Marcus Eriksson marc...@apache.org
Authored: Wed Mar 12 14:04:47 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Wed Mar 12 14:06:05 2014 +0100

--
 CHANGES.txt |   1 +
 pylib/cqlshlib/cql3handling.py  |  13 +-
 .../apache/cassandra/cache/CachingOptions.java  | 288 +++
 .../org/apache/cassandra/cli/CliClient.java |   3 +-
 .../org/apache/cassandra/config/CFMetaData.java | 110 ++-
 .../cassandra/cql/AlterTableStatement.java  |   4 +-
 .../cql/CreateColumnFamilyStatement.java|   4 +-
 .../cassandra/cql3/statements/CFPropDefs.java   |  25 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |  18 +-
 .../org/apache/cassandra/db/SystemKeyspace.java |  33 ++-
 .../cassandra/io/sstable/SSTableReader.java |   6 +-
 .../unit/org/apache/cassandra/SchemaLoader.java |  14 +-
 12 files changed, 391 insertions(+), 128 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/da444a69/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index d8eb3a1..61e17e3 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -11,6 +11,7 @@
  * Improve handling of range tombstone for wide partitions (CASSANDRA-6446)
  * Fix ClassCastException for compact table with composites (CASSANDRA-6738)
  * Fix potentially repairing with wrong nodes (CASSANDRA-6808)
+ * Change caching option syntax (CASSANDRA-6745)
 Merged from 2.0:
  * Fix saving triggers to schema (CASSANDRA-6789)
  * Fix trigger mutations when base mutation list is immutable (CASSANDRA-6790)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/da444a69/pylib/cqlshlib/cql3handling.py
--
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 158e60d..3522d1c 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -62,8 +62,6 @@ class Cql3ParsingRuleSet(CqlParsingRuleSet):
 
 columnfamily_layout_options = (
 ('bloom_filter_fp_chance', None),
-('caching', None),
-('rows_per_partition_to_cache', None),
 ('comment', None),
 ('dclocal_read_repair_chance', 'local_read_repair_chance'),
 ('gc_grace_seconds', None),
@@ -83,6 +81,8 @@ class Cql3ParsingRuleSet(CqlParsingRuleSet):
 ('class', 'min_threshold', 'max_threshold')),
 ('compression', 'compression_parameters',
 ('sstable_compression', 'chunk_length_kb', 'crc_check_chance')),
+('caching', None,
+('rows_per_partition', 'keys')),
 )
 
 obsolete_cf_options = ()
@@ -463,6 +463,8 @@ def cf_prop_val_mapkey_completer(ctxt, cass):
 pairsseen = dict(zip(keysseen, valsseen))
 if optname == 'compression':
 return map(escape_value, set(subopts).difference(keysseen))
+if optname == 'caching':
+return map(escape_value, set(subopts).difference(keysseen))
 if optname == 'compaction':
 opts = set(subopts)
 try:
@@ -488,6 +490,11 @@ def cf_prop_val_mapval_completer(ctxt, cass):
 if key == 'sstable_compression':
 return map(escape_value, CqlRuleSet.available_compression_classes)
 return [Hint('option_value')]
+elif opt == 'caching':
+if key == 'rows_per_partition':
+return [Hint('ALL'), Hint('NONE'), Hint('#rows_per_partition')]
+elif key == 'keys':
+return [Hint('ALL'), Hint('NONE')]
 return ()
 
 def cf_prop_val_mapender_completer(ctxt, cass):
@@ -1187,7 +1194,7 @@ class CqlTableDef:
 for attr, val in layout.items():
 setattr(cf, attr.encode('ascii'), val)
 cf.comparator = lookup_casstype(cf.comparator)
-for attr in ('compaction_strategy_options', 'compression_parameters'):
+for attr in ('compaction_strategy_options', 'compression_parameters', 
'caching'):
 setattr(cf, attr, json.loads(getattr(cf, attr)))
 
 # deal with columns, filter out empty column names (see CASSANDRA-6139)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/da444a69/src/java/org/apache/cassandra/cache/CachingOptions.java

[1/2] git commit: Update syntax for changing caching options.

2014-03-12 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/trunk a03ac8d07 - 346d8fb6d


Update syntax for changing caching options.

Patch by marcuse, reviewed by slebresne for CASSANDRA-6745.


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

Branch: refs/heads/trunk
Commit: da444a6903ddd8d02e7b251c8b38faf11d3a5c93
Parents: ad5169d
Author: Marcus Eriksson marc...@apache.org
Authored: Wed Mar 12 14:04:47 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Wed Mar 12 14:06:05 2014 +0100

--
 CHANGES.txt |   1 +
 pylib/cqlshlib/cql3handling.py  |  13 +-
 .../apache/cassandra/cache/CachingOptions.java  | 288 +++
 .../org/apache/cassandra/cli/CliClient.java |   3 +-
 .../org/apache/cassandra/config/CFMetaData.java | 110 ++-
 .../cassandra/cql/AlterTableStatement.java  |   4 +-
 .../cql/CreateColumnFamilyStatement.java|   4 +-
 .../cassandra/cql3/statements/CFPropDefs.java   |  25 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |  18 +-
 .../org/apache/cassandra/db/SystemKeyspace.java |  33 ++-
 .../cassandra/io/sstable/SSTableReader.java |   6 +-
 .../unit/org/apache/cassandra/SchemaLoader.java |  14 +-
 12 files changed, 391 insertions(+), 128 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/da444a69/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index d8eb3a1..61e17e3 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -11,6 +11,7 @@
  * Improve handling of range tombstone for wide partitions (CASSANDRA-6446)
  * Fix ClassCastException for compact table with composites (CASSANDRA-6738)
  * Fix potentially repairing with wrong nodes (CASSANDRA-6808)
+ * Change caching option syntax (CASSANDRA-6745)
 Merged from 2.0:
  * Fix saving triggers to schema (CASSANDRA-6789)
  * Fix trigger mutations when base mutation list is immutable (CASSANDRA-6790)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/da444a69/pylib/cqlshlib/cql3handling.py
--
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 158e60d..3522d1c 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -62,8 +62,6 @@ class Cql3ParsingRuleSet(CqlParsingRuleSet):
 
 columnfamily_layout_options = (
 ('bloom_filter_fp_chance', None),
-('caching', None),
-('rows_per_partition_to_cache', None),
 ('comment', None),
 ('dclocal_read_repair_chance', 'local_read_repair_chance'),
 ('gc_grace_seconds', None),
@@ -83,6 +81,8 @@ class Cql3ParsingRuleSet(CqlParsingRuleSet):
 ('class', 'min_threshold', 'max_threshold')),
 ('compression', 'compression_parameters',
 ('sstable_compression', 'chunk_length_kb', 'crc_check_chance')),
+('caching', None,
+('rows_per_partition', 'keys')),
 )
 
 obsolete_cf_options = ()
@@ -463,6 +463,8 @@ def cf_prop_val_mapkey_completer(ctxt, cass):
 pairsseen = dict(zip(keysseen, valsseen))
 if optname == 'compression':
 return map(escape_value, set(subopts).difference(keysseen))
+if optname == 'caching':
+return map(escape_value, set(subopts).difference(keysseen))
 if optname == 'compaction':
 opts = set(subopts)
 try:
@@ -488,6 +490,11 @@ def cf_prop_val_mapval_completer(ctxt, cass):
 if key == 'sstable_compression':
 return map(escape_value, CqlRuleSet.available_compression_classes)
 return [Hint('option_value')]
+elif opt == 'caching':
+if key == 'rows_per_partition':
+return [Hint('ALL'), Hint('NONE'), Hint('#rows_per_partition')]
+elif key == 'keys':
+return [Hint('ALL'), Hint('NONE')]
 return ()
 
 def cf_prop_val_mapender_completer(ctxt, cass):
@@ -1187,7 +1194,7 @@ class CqlTableDef:
 for attr, val in layout.items():
 setattr(cf, attr.encode('ascii'), val)
 cf.comparator = lookup_casstype(cf.comparator)
-for attr in ('compaction_strategy_options', 'compression_parameters'):
+for attr in ('compaction_strategy_options', 'compression_parameters', 
'caching'):
 setattr(cf, attr, json.loads(getattr(cf, attr)))
 
 # deal with columns, filter out empty column names (see CASSANDRA-6139)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/da444a69/src/java/org/apache/cassandra/cache/CachingOptions.java

[2/2] git commit: Merge branch 'cassandra-2.1' into trunk

2014-03-12 Thread marcuse
Merge branch 'cassandra-2.1' into trunk

Conflicts:
src/java/org/apache/cassandra/cql/AlterTableStatement.java
src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java


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

Branch: refs/heads/trunk
Commit: 346d8fb6dd99c86b632a0b99a368231bbd34b0f7
Parents: a03ac8d da444a6
Author: Marcus Eriksson marc...@apache.org
Authored: Wed Mar 12 14:09:16 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Wed Mar 12 14:09:16 2014 +0100

--
 CHANGES.txt |   1 +
 pylib/cqlshlib/cql3handling.py  |  13 +-
 .../apache/cassandra/cache/CachingOptions.java  | 288 +++
 .../org/apache/cassandra/cli/CliClient.java |   3 +-
 .../org/apache/cassandra/config/CFMetaData.java | 110 ++-
 .../cassandra/cql3/statements/CFPropDefs.java   |  25 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |  18 +-
 .../org/apache/cassandra/db/SystemKeyspace.java |  33 ++-
 .../cassandra/io/sstable/SSTableReader.java |   6 +-
 .../unit/org/apache/cassandra/SchemaLoader.java |  14 +-
 10 files changed, 387 insertions(+), 124 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/346d8fb6/CHANGES.txt
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/346d8fb6/src/java/org/apache/cassandra/config/CFMetaData.java
--



git commit: fixed bug in CompactionManager.needsCleanup

2014-03-12 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.0 19444ea16 - 05da045bb


fixed bug in CompactionManager.needsCleanup

Patch by Dmitrij Koniajev, reviewed by marcuse for CASSANDRA-6845.


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

Branch: refs/heads/cassandra-2.0
Commit: 05da045bbfe29bf472982d3c5da90050176202df
Parents: 19444ea
Author: Dmitrij Koniajev dimchan...@gmail.com
Authored: Wed Mar 12 17:05:31 2014 +0200
Committer: Marcus Eriksson marc...@apache.org
Committed: Wed Mar 12 18:14:59 2014 +0100

--
 CHANGES.txt| 2 +-
 src/java/org/apache/cassandra/db/compaction/CompactionManager.java | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/05da045b/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 42dc4ee..3531737 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,7 +4,7 @@
  * Fix accounting in FileCacheService to allow re-using RAR (CASSANDRA-6838)
  * Fix static counter columns (CASSANDRA-6827)
  * Restore expiring-deleted (cell) compaction optimization (CASSANDRA-6844)
-
+ * Fix CompactionManager.needsCleanup (CASSANDRA-6845)
 
 2.0.6
  * Avoid race-prone second scrub of system keyspace (CASSANDRA-6797)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/05da045b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
--
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java 
b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 02cf433..453176e 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -493,7 +493,7 @@ public class CompactionManager implements 
CompactionManagerMBean
 return false;
 }
 
-if (i == (ownedRanges.size() - 1))
+if (i == (sortedRanges.size() - 1))
 {
 // we're at the last range and we found a key beyond the end 
of the range
 return true;



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

2014-03-12 Thread marcuse
Merge branch 'cassandra-2.0' into cassandra-2.1

Conflicts:
CHANGES.txt


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

Branch: refs/heads/cassandra-2.1
Commit: 019e338670ecbf84fab9ae9814db0c5fcf2f1dff
Parents: cf9ae5f 05da045
Author: Marcus Eriksson marc...@apache.org
Authored: Wed Mar 12 18:17:22 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Wed Mar 12 18:17:22 2014 +0100

--
 CHANGES.txt| 1 +
 src/java/org/apache/cassandra/db/compaction/CompactionManager.java | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/019e3386/CHANGES.txt
--
diff --cc CHANGES.txt
index cb51c12,3531737..00cf087
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -20,6 -4,9 +20,7 @@@ Merged from 2.0
   * Fix accounting in FileCacheService to allow re-using RAR (CASSANDRA-6838)
   * Fix static counter columns (CASSANDRA-6827)
   * Restore expiring-deleted (cell) compaction optimization (CASSANDRA-6844)
+  * Fix CompactionManager.needsCleanup (CASSANDRA-6845)
 -
 -2.0.6
   * Avoid race-prone second scrub of system keyspace (CASSANDRA-6797)
   * Pool CqlRecordWriter clients by inetaddress rather than Range 
 (CASSANDRA-6665)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/019e3386/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
--



[1/2] git commit: fixed bug in CompactionManager.needsCleanup

2014-03-12 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 cf9ae5f9c - 019e33867


fixed bug in CompactionManager.needsCleanup

Patch by Dmitrij Koniajev, reviewed by marcuse for CASSANDRA-6845.


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

Branch: refs/heads/cassandra-2.1
Commit: 05da045bbfe29bf472982d3c5da90050176202df
Parents: 19444ea
Author: Dmitrij Koniajev dimchan...@gmail.com
Authored: Wed Mar 12 17:05:31 2014 +0200
Committer: Marcus Eriksson marc...@apache.org
Committed: Wed Mar 12 18:14:59 2014 +0100

--
 CHANGES.txt| 2 +-
 src/java/org/apache/cassandra/db/compaction/CompactionManager.java | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/05da045b/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 42dc4ee..3531737 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,7 +4,7 @@
  * Fix accounting in FileCacheService to allow re-using RAR (CASSANDRA-6838)
  * Fix static counter columns (CASSANDRA-6827)
  * Restore expiring-deleted (cell) compaction optimization (CASSANDRA-6844)
-
+ * Fix CompactionManager.needsCleanup (CASSANDRA-6845)
 
 2.0.6
  * Avoid race-prone second scrub of system keyspace (CASSANDRA-6797)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/05da045b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
--
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java 
b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 02cf433..453176e 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -493,7 +493,7 @@ public class CompactionManager implements 
CompactionManagerMBean
 return false;
 }
 
-if (i == (ownedRanges.size() - 1))
+if (i == (sortedRanges.size() - 1))
 {
 // we're at the last range and we found a key beyond the end 
of the range
 return true;



[1/3] git commit: fixed bug in CompactionManager.needsCleanup

2014-03-12 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/trunk c4b2320ae - 4f525b3a8


fixed bug in CompactionManager.needsCleanup

Patch by Dmitrij Koniajev, reviewed by marcuse for CASSANDRA-6845.


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

Branch: refs/heads/trunk
Commit: 05da045bbfe29bf472982d3c5da90050176202df
Parents: 19444ea
Author: Dmitrij Koniajev dimchan...@gmail.com
Authored: Wed Mar 12 17:05:31 2014 +0200
Committer: Marcus Eriksson marc...@apache.org
Committed: Wed Mar 12 18:14:59 2014 +0100

--
 CHANGES.txt| 2 +-
 src/java/org/apache/cassandra/db/compaction/CompactionManager.java | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/05da045b/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 42dc4ee..3531737 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,7 +4,7 @@
  * Fix accounting in FileCacheService to allow re-using RAR (CASSANDRA-6838)
  * Fix static counter columns (CASSANDRA-6827)
  * Restore expiring-deleted (cell) compaction optimization (CASSANDRA-6844)
-
+ * Fix CompactionManager.needsCleanup (CASSANDRA-6845)
 
 2.0.6
  * Avoid race-prone second scrub of system keyspace (CASSANDRA-6797)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/05da045b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
--
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java 
b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 02cf433..453176e 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -493,7 +493,7 @@ public class CompactionManager implements 
CompactionManagerMBean
 return false;
 }
 
-if (i == (ownedRanges.size() - 1))
+if (i == (sortedRanges.size() - 1))
 {
 // we're at the last range and we found a key beyond the end 
of the range
 return true;



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

2014-03-12 Thread marcuse
Merge branch 'cassandra-2.0' into cassandra-2.1

Conflicts:
CHANGES.txt


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

Branch: refs/heads/trunk
Commit: 019e338670ecbf84fab9ae9814db0c5fcf2f1dff
Parents: cf9ae5f 05da045
Author: Marcus Eriksson marc...@apache.org
Authored: Wed Mar 12 18:17:22 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Wed Mar 12 18:17:22 2014 +0100

--
 CHANGES.txt| 1 +
 src/java/org/apache/cassandra/db/compaction/CompactionManager.java | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/019e3386/CHANGES.txt
--
diff --cc CHANGES.txt
index cb51c12,3531737..00cf087
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -20,6 -4,9 +20,7 @@@ Merged from 2.0
   * Fix accounting in FileCacheService to allow re-using RAR (CASSANDRA-6838)
   * Fix static counter columns (CASSANDRA-6827)
   * Restore expiring-deleted (cell) compaction optimization (CASSANDRA-6844)
+  * Fix CompactionManager.needsCleanup (CASSANDRA-6845)
 -
 -2.0.6
   * Avoid race-prone second scrub of system keyspace (CASSANDRA-6797)
   * Pool CqlRecordWriter clients by inetaddress rather than Range 
 (CASSANDRA-6665)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/019e3386/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
--



[3/3] git commit: Merge branch 'cassandra-2.1' into trunk

2014-03-12 Thread marcuse
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 4f525b3a804ee600ef879f3c1b870b134806aea9
Parents: c4b2320 019e338
Author: Marcus Eriksson marc...@apache.org
Authored: Wed Mar 12 18:17:43 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Wed Mar 12 18:17:43 2014 +0100

--
 CHANGES.txt| 1 +
 src/java/org/apache/cassandra/db/compaction/CompactionManager.java | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4f525b3a/CHANGES.txt
--



git commit: Read message id as string from older nodes.

2014-03-14 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.0 c49d33633 - c3aac4560


Read message id as string from older nodes.

Patch by marcuse, reviewed by jbellis for CASSANDRA-6840


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

Branch: refs/heads/cassandra-2.0
Commit: c3aac45605416251f7bb9a3260299f1c4416cb89
Parents: c49d336
Author: Marcus Eriksson marc...@apache.org
Authored: Fri Mar 14 18:51:01 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Fri Mar 14 18:52:59 2014 +0100

--
 CHANGES.txt  |  1 +
 .../apache/cassandra/db/RowMutationVerbHandler.java  | 15 ---
 2 files changed, 13 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3aac456/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index e208e21..2b8dea8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@
  * Restore expiring-deleted (cell) compaction optimization (CASSANDRA-6844)
  * Fix CompactionManager.needsCleanup (CASSANDRA-6845)
  * Correctly compare BooleanType values other than 0 and 1 (CASSANDRA-6779)
+ * Read message id as string from earlier versions (CASSANDRA-6840)
 Merged from 1.2:
  * fix nodetool getsstables for blob PK (CASSANDRA-6803)
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3aac456/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java
--
diff --git a/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java 
b/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java
index dcdfc2e..da7fa6a 100644
--- a/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java
@@ -46,7 +46,7 @@ public class RowMutationVerbHandler implements 
IVerbHandlerRowMutation
 replyTo = message.from;
 byte[] forwardBytes = 
message.parameters.get(RowMutation.FORWARD_TO);
 if (forwardBytes != null)
-forwardToLocalNodes(rm, message.verb, forwardBytes, 
message.from);
+forwardToLocalNodes(rm, message.verb, forwardBytes, 
message.from, message.version);
 }
 else
 {
@@ -68,7 +68,7 @@ public class RowMutationVerbHandler implements 
IVerbHandlerRowMutation
  * Older version ( 1.0) will not send this message at all, hence we don't
  * need to check the version of the data.
  */
-private void forwardToLocalNodes(RowMutation rm, MessagingService.Verb 
verb, byte[] forwardBytes, InetAddress from) throws IOException
+private void forwardToLocalNodes(RowMutation rm, MessagingService.Verb 
verb, byte[] forwardBytes, InetAddress from, int version) throws IOException
 {
 DataInputStream in = new DataInputStream(new 
FastByteArrayInputStream(forwardBytes));
 int size = in.readInt();
@@ -79,7 +79,16 @@ public class RowMutationVerbHandler implements 
IVerbHandlerRowMutation
 for (int i = 0; i  size; i++)
 {
 InetAddress address = 
CompactEndpointSerializationHelper.deserialize(in);
-int id = in.readInt();
+int id;
+if (version  MessagingService.VERSION_20)
+{
+String s = in.readUTF();
+id = Integer.parseInt(s);
+}
+else
+{
+id = in.readInt();
+}
 Tracing.trace(Enqueuing forwarded write to {}, address);
 MessagingService.instance().sendOneWay(message, id, address);
 }



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

2014-03-14 Thread marcuse
Merge branch 'cassandra-2.0' into cassandra-2.1


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

Branch: refs/heads/cassandra-2.1
Commit: 74ac6e8962298f4f04c18803d281293a594b7cee
Parents: ace0e6e c3aac45
Author: Marcus Eriksson marc...@apache.org
Authored: Fri Mar 14 18:53:48 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Fri Mar 14 18:53:48 2014 +0100

--

--




[1/2] git commit: Read message id as string from older nodes.

2014-03-14 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 ace0e6e1d - 74ac6e896


Read message id as string from older nodes.

Patch by marcuse, reviewed by jbellis for CASSANDRA-6840


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

Branch: refs/heads/cassandra-2.1
Commit: c3aac45605416251f7bb9a3260299f1c4416cb89
Parents: c49d336
Author: Marcus Eriksson marc...@apache.org
Authored: Fri Mar 14 18:51:01 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Fri Mar 14 18:52:59 2014 +0100

--
 CHANGES.txt  |  1 +
 .../apache/cassandra/db/RowMutationVerbHandler.java  | 15 ---
 2 files changed, 13 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3aac456/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index e208e21..2b8dea8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@
  * Restore expiring-deleted (cell) compaction optimization (CASSANDRA-6844)
  * Fix CompactionManager.needsCleanup (CASSANDRA-6845)
  * Correctly compare BooleanType values other than 0 and 1 (CASSANDRA-6779)
+ * Read message id as string from earlier versions (CASSANDRA-6840)
 Merged from 1.2:
  * fix nodetool getsstables for blob PK (CASSANDRA-6803)
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3aac456/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java
--
diff --git a/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java 
b/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java
index dcdfc2e..da7fa6a 100644
--- a/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java
@@ -46,7 +46,7 @@ public class RowMutationVerbHandler implements 
IVerbHandlerRowMutation
 replyTo = message.from;
 byte[] forwardBytes = 
message.parameters.get(RowMutation.FORWARD_TO);
 if (forwardBytes != null)
-forwardToLocalNodes(rm, message.verb, forwardBytes, 
message.from);
+forwardToLocalNodes(rm, message.verb, forwardBytes, 
message.from, message.version);
 }
 else
 {
@@ -68,7 +68,7 @@ public class RowMutationVerbHandler implements 
IVerbHandlerRowMutation
  * Older version ( 1.0) will not send this message at all, hence we don't
  * need to check the version of the data.
  */
-private void forwardToLocalNodes(RowMutation rm, MessagingService.Verb 
verb, byte[] forwardBytes, InetAddress from) throws IOException
+private void forwardToLocalNodes(RowMutation rm, MessagingService.Verb 
verb, byte[] forwardBytes, InetAddress from, int version) throws IOException
 {
 DataInputStream in = new DataInputStream(new 
FastByteArrayInputStream(forwardBytes));
 int size = in.readInt();
@@ -79,7 +79,16 @@ public class RowMutationVerbHandler implements 
IVerbHandlerRowMutation
 for (int i = 0; i  size; i++)
 {
 InetAddress address = 
CompactEndpointSerializationHelper.deserialize(in);
-int id = in.readInt();
+int id;
+if (version  MessagingService.VERSION_20)
+{
+String s = in.readUTF();
+id = Integer.parseInt(s);
+}
+else
+{
+id = in.readInt();
+}
 Tracing.trace(Enqueuing forwarded write to {}, address);
 MessagingService.instance().sendOneWay(message, id, address);
 }



[3/3] git commit: Merge branch 'cassandra-2.1' into trunk

2014-03-14 Thread marcuse
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 59695a2541dcc088f3cac1ddb4b65143615a068f
Parents: 79e3217 74ac6e8
Author: Marcus Eriksson marc...@apache.org
Authored: Fri Mar 14 18:54:19 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Fri Mar 14 18:54:19 2014 +0100

--

--




[1/3] git commit: Read message id as string from older nodes.

2014-03-14 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/trunk 79e3217f9 - 59695a254


Read message id as string from older nodes.

Patch by marcuse, reviewed by jbellis for CASSANDRA-6840


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

Branch: refs/heads/trunk
Commit: c3aac45605416251f7bb9a3260299f1c4416cb89
Parents: c49d336
Author: Marcus Eriksson marc...@apache.org
Authored: Fri Mar 14 18:51:01 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Fri Mar 14 18:52:59 2014 +0100

--
 CHANGES.txt  |  1 +
 .../apache/cassandra/db/RowMutationVerbHandler.java  | 15 ---
 2 files changed, 13 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3aac456/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index e208e21..2b8dea8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@
  * Restore expiring-deleted (cell) compaction optimization (CASSANDRA-6844)
  * Fix CompactionManager.needsCleanup (CASSANDRA-6845)
  * Correctly compare BooleanType values other than 0 and 1 (CASSANDRA-6779)
+ * Read message id as string from earlier versions (CASSANDRA-6840)
 Merged from 1.2:
  * fix nodetool getsstables for blob PK (CASSANDRA-6803)
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3aac456/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java
--
diff --git a/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java 
b/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java
index dcdfc2e..da7fa6a 100644
--- a/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java
@@ -46,7 +46,7 @@ public class RowMutationVerbHandler implements 
IVerbHandlerRowMutation
 replyTo = message.from;
 byte[] forwardBytes = 
message.parameters.get(RowMutation.FORWARD_TO);
 if (forwardBytes != null)
-forwardToLocalNodes(rm, message.verb, forwardBytes, 
message.from);
+forwardToLocalNodes(rm, message.verb, forwardBytes, 
message.from, message.version);
 }
 else
 {
@@ -68,7 +68,7 @@ public class RowMutationVerbHandler implements 
IVerbHandlerRowMutation
  * Older version ( 1.0) will not send this message at all, hence we don't
  * need to check the version of the data.
  */
-private void forwardToLocalNodes(RowMutation rm, MessagingService.Verb 
verb, byte[] forwardBytes, InetAddress from) throws IOException
+private void forwardToLocalNodes(RowMutation rm, MessagingService.Verb 
verb, byte[] forwardBytes, InetAddress from, int version) throws IOException
 {
 DataInputStream in = new DataInputStream(new 
FastByteArrayInputStream(forwardBytes));
 int size = in.readInt();
@@ -79,7 +79,16 @@ public class RowMutationVerbHandler implements 
IVerbHandlerRowMutation
 for (int i = 0; i  size; i++)
 {
 InetAddress address = 
CompactEndpointSerializationHelper.deserialize(in);
-int id = in.readInt();
+int id;
+if (version  MessagingService.VERSION_20)
+{
+String s = in.readUTF();
+id = Integer.parseInt(s);
+}
+else
+{
+id = in.readInt();
+}
 Tracing.trace(Enqueuing forwarded write to {}, address);
 MessagingService.instance().sendOneWay(message, id, address);
 }



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

2014-03-14 Thread marcuse
Merge branch 'cassandra-2.0' into cassandra-2.1


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

Branch: refs/heads/trunk
Commit: 74ac6e8962298f4f04c18803d281293a594b7cee
Parents: ace0e6e c3aac45
Author: Marcus Eriksson marc...@apache.org
Authored: Fri Mar 14 18:53:48 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Fri Mar 14 18:53:48 2014 +0100

--

--




git commit: Add paranoid disk failure option.

2014-03-18 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.0 b4f2ff17a - 850cd59ca


Add paranoid disk failure option.

Patch by marcuse, reviewed by kohlisankalp and jbellis for CASSANDRA-6646


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

Branch: refs/heads/cassandra-2.0
Commit: 850cd59ca4f409a9c9a2dc0edd7bf02f52308a35
Parents: b4f2ff1
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Mar 18 07:52:12 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Tue Mar 18 07:52:30 2014 +0100

--
 CHANGES.txt| 1 +
 conf/cassandra.yaml| 1 +
 src/java/org/apache/cassandra/config/Config.java   | 1 +
 src/java/org/apache/cassandra/io/util/FileUtils.java   | 8 
 src/java/org/apache/cassandra/service/CassandraDaemon.java | 9 +
 5 files changed, 20 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/850cd59c/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 040af7c..86d4e6f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -14,6 +14,7 @@
  * Correctly compare BooleanType values other than 0 and 1 (CASSANDRA-6779)
  * Read message id as string from earlier versions (CASSANDRA-6840)
  * Properly use the Paxos consistency for (non-protocol) batch (CASSANDRA-6837)
+ * Add paranoid disk failure option (CASSANDRA-6646)
 Merged from 1.2:
  * add extra SSL cipher suites (CASSANDRA-6613)
  * fix nodetool getsstables for blob PK (CASSANDRA-6803)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/850cd59c/conf/cassandra.yaml
--
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index dc71c0f..d3c91ef 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -111,6 +111,7 @@ data_file_directories:
 commitlog_directory: /var/lib/cassandra/commitlog
 
 # policy for data disk failures:
+# stop_paranoid: shut down gossip and Thrift even for single-sstable errors.
 # stop: shut down gossip and Thrift, leaving the node effectively dead, but
 #   can still be inspected via JMX.
 # best_effort: stop using the failed disk and respond to requests based on

http://git-wip-us.apache.org/repos/asf/cassandra/blob/850cd59c/src/java/org/apache/cassandra/config/Config.java
--
diff --git a/src/java/org/apache/cassandra/config/Config.java 
b/src/java/org/apache/cassandra/config/Config.java
index e8e43e2..5317fb8 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -275,6 +275,7 @@ public class Config
 best_effort,
 stop,
 ignore,
+stop_paranoid,
 }
 
 public static enum CommitFailurePolicy

http://git-wip-us.apache.org/repos/asf/cassandra/blob/850cd59c/src/java/org/apache/cassandra/io/util/FileUtils.java
--
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java 
b/src/java/org/apache/cassandra/io/util/FileUtils.java
index e091465..a12745c 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -38,6 +38,7 @@ import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.service.StorageService;
 
 public class FileUtils
@@ -407,11 +408,18 @@ public class FileUtils
 n += skipped;
 }
 }
+
+public static void handleCorruptSSTable(CorruptSSTableException e)
+{
+if (DatabaseDescriptor.getDiskFailurePolicy() == 
Config.DiskFailurePolicy.stop_paranoid)
+StorageService.instance.stopTransports();
+}
 
 public static void handleFSError(FSError e)
 {
 switch (DatabaseDescriptor.getDiskFailurePolicy())
 {
+case stop_paranoid:
 case stop:
 StorageService.instance.stopTransports();
 break;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/850cd59c/src/java/org/apache/cassandra/service/CassandraDaemon.java
--
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java 
b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index d8c56f1

[1/2] git commit: Add paranoid disk failure option.

2014-03-18 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 815892e8c - 35b215136


Add paranoid disk failure option.

Patch by marcuse, reviewed by kohlisankalp and jbellis for CASSANDRA-6646


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

Branch: refs/heads/cassandra-2.1
Commit: 850cd59ca4f409a9c9a2dc0edd7bf02f52308a35
Parents: b4f2ff1
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Mar 18 07:52:12 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Tue Mar 18 07:52:30 2014 +0100

--
 CHANGES.txt| 1 +
 conf/cassandra.yaml| 1 +
 src/java/org/apache/cassandra/config/Config.java   | 1 +
 src/java/org/apache/cassandra/io/util/FileUtils.java   | 8 
 src/java/org/apache/cassandra/service/CassandraDaemon.java | 9 +
 5 files changed, 20 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/850cd59c/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 040af7c..86d4e6f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -14,6 +14,7 @@
  * Correctly compare BooleanType values other than 0 and 1 (CASSANDRA-6779)
  * Read message id as string from earlier versions (CASSANDRA-6840)
  * Properly use the Paxos consistency for (non-protocol) batch (CASSANDRA-6837)
+ * Add paranoid disk failure option (CASSANDRA-6646)
 Merged from 1.2:
  * add extra SSL cipher suites (CASSANDRA-6613)
  * fix nodetool getsstables for blob PK (CASSANDRA-6803)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/850cd59c/conf/cassandra.yaml
--
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index dc71c0f..d3c91ef 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -111,6 +111,7 @@ data_file_directories:
 commitlog_directory: /var/lib/cassandra/commitlog
 
 # policy for data disk failures:
+# stop_paranoid: shut down gossip and Thrift even for single-sstable errors.
 # stop: shut down gossip and Thrift, leaving the node effectively dead, but
 #   can still be inspected via JMX.
 # best_effort: stop using the failed disk and respond to requests based on

http://git-wip-us.apache.org/repos/asf/cassandra/blob/850cd59c/src/java/org/apache/cassandra/config/Config.java
--
diff --git a/src/java/org/apache/cassandra/config/Config.java 
b/src/java/org/apache/cassandra/config/Config.java
index e8e43e2..5317fb8 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -275,6 +275,7 @@ public class Config
 best_effort,
 stop,
 ignore,
+stop_paranoid,
 }
 
 public static enum CommitFailurePolicy

http://git-wip-us.apache.org/repos/asf/cassandra/blob/850cd59c/src/java/org/apache/cassandra/io/util/FileUtils.java
--
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java 
b/src/java/org/apache/cassandra/io/util/FileUtils.java
index e091465..a12745c 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -38,6 +38,7 @@ import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.service.StorageService;
 
 public class FileUtils
@@ -407,11 +408,18 @@ public class FileUtils
 n += skipped;
 }
 }
+
+public static void handleCorruptSSTable(CorruptSSTableException e)
+{
+if (DatabaseDescriptor.getDiskFailurePolicy() == 
Config.DiskFailurePolicy.stop_paranoid)
+StorageService.instance.stopTransports();
+}
 
 public static void handleFSError(FSError e)
 {
 switch (DatabaseDescriptor.getDiskFailurePolicy())
 {
+case stop_paranoid:
 case stop:
 StorageService.instance.stopTransports();
 break;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/850cd59c/src/java/org/apache/cassandra/service/CassandraDaemon.java
--
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java 
b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index d8c56f1

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

2014-03-18 Thread marcuse
Merge branch 'cassandra-2.0' into cassandra-2.1

Conflicts:
CHANGES.txt
src/java/org/apache/cassandra/service/CassandraDaemon.java


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

Branch: refs/heads/trunk
Commit: 35b21513697f584ce45dbe99e62b098fc3db957e
Parents: 815892e 850cd59
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Mar 18 08:08:00 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Tue Mar 18 08:09:02 2014 +0100

--
 CHANGES.txt |  1 +
 conf/cassandra.yaml |  1 +
 .../org/apache/cassandra/config/Config.java |  1 +
 .../org/apache/cassandra/io/util/FileUtils.java | 27 
 .../cassandra/service/CassandraDaemon.java  |  9 +++
 .../cassandra/service/StorageService.java   | 19 ++
 6 files changed, 41 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/35b21513/CHANGES.txt
--
diff --cc CHANGES.txt
index f6295ae,86d4e6f..04e16a0
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -39,6 -14,8 +39,7 @@@ Merged from 2.0
   * Correctly compare BooleanType values other than 0 and 1 (CASSANDRA-6779)
   * Read message id as string from earlier versions (CASSANDRA-6840)
   * Properly use the Paxos consistency for (non-protocol) batch 
(CASSANDRA-6837)
+  * Add paranoid disk failure option (CASSANDRA-6646)
 -Merged from 1.2:
   * add extra SSL cipher suites (CASSANDRA-6613)
   * fix nodetool getsstables for blob PK (CASSANDRA-6803)
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35b21513/conf/cassandra.yaml
--
diff --cc conf/cassandra.yaml
index 0193875,d3c91ef..fc525e4
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@@ -107,16 -107,11 +107,17 @@@ partitioner: org.apache.cassandra.dht.M
  data_file_directories:
  - /var/lib/cassandra/data
  
 -# commit log
 +# commit log.  when running on magnetic HDD, this should be a
 +# separate spindle than the data directories.
  commitlog_directory: /var/lib/cassandra/commitlog
  
 +# location to write flushing sstables to.  Ideally, this will also be
 +# a separate spindle in HDD deployments.  If you only have two spindles,
 +# have it share with the data spindle.
 +flush_directory: /var/lib/cassandra/flush
 +
  # policy for data disk failures:
+ # stop_paranoid: shut down gossip and Thrift even for single-sstable errors.
  # stop: shut down gossip and Thrift, leaving the node effectively dead, but
  #   can still be inspected via JMX.
  # best_effort: stop using the failed disk and respond to requests based on

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35b21513/src/java/org/apache/cassandra/config/Config.java
--
diff --cc src/java/org/apache/cassandra/config/Config.java
index c5d35a3,5317fb8..2ce22f3
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@@ -292,8 -275,16 +292,9 @@@ public class Confi
  best_effort,
  stop,
  ignore,
+ stop_paranoid,
  }
  
 -public static enum CommitFailurePolicy
 -{
 -stop,
 -stop_commit,
 -ignore,
 -}
 -
  public static enum RequestSchedulerId
  {
  keyspace

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35b21513/src/java/org/apache/cassandra/io/util/FileUtils.java
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35b21513/src/java/org/apache/cassandra/service/CassandraDaemon.java
--
diff --cc src/java/org/apache/cassandra/service/CassandraDaemon.java
index 7c56fb7,0970776..7259a17
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@@ -32,8 -33,12 +32,10 @@@ import javax.management.ObjectName
  import javax.management.StandardMBean;
  
  import com.addthis.metrics.reporter.config.ReporterConfig;
 -
  import com.google.common.collect.Iterables;
  import com.google.common.util.concurrent.Uninterruptibles;
+ 
+ import org.apache.cassandra.io.sstable.CorruptSSTableException;
 -import org.apache.log4j.PropertyConfigurator;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
@@@ -173,9 -206,16 +175,16 @@@ public class CassandraDaemo
  if (e2 instanceof FSError)
  

[3/3] git commit: Merge branch 'cassandra-2.1' into trunk

2014-03-18 Thread marcuse
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 6e66cf5f2a40c7d24848b081020cb1abffb10157
Parents: f3b2a6b 35b2151
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Mar 18 08:10:42 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Tue Mar 18 08:10:42 2014 +0100

--
 CHANGES.txt |  1 +
 conf/cassandra.yaml |  1 +
 .../org/apache/cassandra/config/Config.java |  1 +
 .../org/apache/cassandra/io/util/FileUtils.java | 27 
 .../cassandra/service/CassandraDaemon.java  |  9 +++
 .../cassandra/service/StorageService.java   | 19 ++
 6 files changed, 41 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6e66cf5f/CHANGES.txt
--



[1/3] git commit: Add paranoid disk failure option.

2014-03-18 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/trunk f3b2a6bcb - 6e66cf5f2


Add paranoid disk failure option.

Patch by marcuse, reviewed by kohlisankalp and jbellis for CASSANDRA-6646


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

Branch: refs/heads/trunk
Commit: 850cd59ca4f409a9c9a2dc0edd7bf02f52308a35
Parents: b4f2ff1
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Mar 18 07:52:12 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Tue Mar 18 07:52:30 2014 +0100

--
 CHANGES.txt| 1 +
 conf/cassandra.yaml| 1 +
 src/java/org/apache/cassandra/config/Config.java   | 1 +
 src/java/org/apache/cassandra/io/util/FileUtils.java   | 8 
 src/java/org/apache/cassandra/service/CassandraDaemon.java | 9 +
 5 files changed, 20 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/850cd59c/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 040af7c..86d4e6f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -14,6 +14,7 @@
  * Correctly compare BooleanType values other than 0 and 1 (CASSANDRA-6779)
  * Read message id as string from earlier versions (CASSANDRA-6840)
  * Properly use the Paxos consistency for (non-protocol) batch (CASSANDRA-6837)
+ * Add paranoid disk failure option (CASSANDRA-6646)
 Merged from 1.2:
  * add extra SSL cipher suites (CASSANDRA-6613)
  * fix nodetool getsstables for blob PK (CASSANDRA-6803)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/850cd59c/conf/cassandra.yaml
--
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index dc71c0f..d3c91ef 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -111,6 +111,7 @@ data_file_directories:
 commitlog_directory: /var/lib/cassandra/commitlog
 
 # policy for data disk failures:
+# stop_paranoid: shut down gossip and Thrift even for single-sstable errors.
 # stop: shut down gossip and Thrift, leaving the node effectively dead, but
 #   can still be inspected via JMX.
 # best_effort: stop using the failed disk and respond to requests based on

http://git-wip-us.apache.org/repos/asf/cassandra/blob/850cd59c/src/java/org/apache/cassandra/config/Config.java
--
diff --git a/src/java/org/apache/cassandra/config/Config.java 
b/src/java/org/apache/cassandra/config/Config.java
index e8e43e2..5317fb8 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -275,6 +275,7 @@ public class Config
 best_effort,
 stop,
 ignore,
+stop_paranoid,
 }
 
 public static enum CommitFailurePolicy

http://git-wip-us.apache.org/repos/asf/cassandra/blob/850cd59c/src/java/org/apache/cassandra/io/util/FileUtils.java
--
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java 
b/src/java/org/apache/cassandra/io/util/FileUtils.java
index e091465..a12745c 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -38,6 +38,7 @@ import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.service.StorageService;
 
 public class FileUtils
@@ -407,11 +408,18 @@ public class FileUtils
 n += skipped;
 }
 }
+
+public static void handleCorruptSSTable(CorruptSSTableException e)
+{
+if (DatabaseDescriptor.getDiskFailurePolicy() == 
Config.DiskFailurePolicy.stop_paranoid)
+StorageService.instance.stopTransports();
+}
 
 public static void handleFSError(FSError e)
 {
 switch (DatabaseDescriptor.getDiskFailurePolicy())
 {
+case stop_paranoid:
 case stop:
 StorageService.instance.stopTransports();
 break;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/850cd59c/src/java/org/apache/cassandra/service/CassandraDaemon.java
--
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java 
b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index d8c56f1..0970776 100644

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

2014-03-18 Thread marcuse
Merge branch 'cassandra-2.0' into cassandra-2.1

Conflicts:
CHANGES.txt
src/java/org/apache/cassandra/service/CassandraDaemon.java


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

Branch: refs/heads/cassandra-2.1
Commit: 35b21513697f584ce45dbe99e62b098fc3db957e
Parents: 815892e 850cd59
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Mar 18 08:08:00 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Tue Mar 18 08:09:02 2014 +0100

--
 CHANGES.txt |  1 +
 conf/cassandra.yaml |  1 +
 .../org/apache/cassandra/config/Config.java |  1 +
 .../org/apache/cassandra/io/util/FileUtils.java | 27 
 .../cassandra/service/CassandraDaemon.java  |  9 +++
 .../cassandra/service/StorageService.java   | 19 ++
 6 files changed, 41 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/35b21513/CHANGES.txt
--
diff --cc CHANGES.txt
index f6295ae,86d4e6f..04e16a0
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -39,6 -14,8 +39,7 @@@ Merged from 2.0
   * Correctly compare BooleanType values other than 0 and 1 (CASSANDRA-6779)
   * Read message id as string from earlier versions (CASSANDRA-6840)
   * Properly use the Paxos consistency for (non-protocol) batch 
(CASSANDRA-6837)
+  * Add paranoid disk failure option (CASSANDRA-6646)
 -Merged from 1.2:
   * add extra SSL cipher suites (CASSANDRA-6613)
   * fix nodetool getsstables for blob PK (CASSANDRA-6803)
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35b21513/conf/cassandra.yaml
--
diff --cc conf/cassandra.yaml
index 0193875,d3c91ef..fc525e4
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@@ -107,16 -107,11 +107,17 @@@ partitioner: org.apache.cassandra.dht.M
  data_file_directories:
  - /var/lib/cassandra/data
  
 -# commit log
 +# commit log.  when running on magnetic HDD, this should be a
 +# separate spindle than the data directories.
  commitlog_directory: /var/lib/cassandra/commitlog
  
 +# location to write flushing sstables to.  Ideally, this will also be
 +# a separate spindle in HDD deployments.  If you only have two spindles,
 +# have it share with the data spindle.
 +flush_directory: /var/lib/cassandra/flush
 +
  # policy for data disk failures:
+ # stop_paranoid: shut down gossip and Thrift even for single-sstable errors.
  # stop: shut down gossip and Thrift, leaving the node effectively dead, but
  #   can still be inspected via JMX.
  # best_effort: stop using the failed disk and respond to requests based on

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35b21513/src/java/org/apache/cassandra/config/Config.java
--
diff --cc src/java/org/apache/cassandra/config/Config.java
index c5d35a3,5317fb8..2ce22f3
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@@ -292,8 -275,16 +292,9 @@@ public class Confi
  best_effort,
  stop,
  ignore,
+ stop_paranoid,
  }
  
 -public static enum CommitFailurePolicy
 -{
 -stop,
 -stop_commit,
 -ignore,
 -}
 -
  public static enum RequestSchedulerId
  {
  keyspace

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35b21513/src/java/org/apache/cassandra/io/util/FileUtils.java
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35b21513/src/java/org/apache/cassandra/service/CassandraDaemon.java
--
diff --cc src/java/org/apache/cassandra/service/CassandraDaemon.java
index 7c56fb7,0970776..7259a17
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@@ -32,8 -33,12 +32,10 @@@ import javax.management.ObjectName
  import javax.management.StandardMBean;
  
  import com.addthis.metrics.reporter.config.ReporterConfig;
 -
  import com.google.common.collect.Iterables;
  import com.google.common.util.concurrent.Uninterruptibles;
+ 
+ import org.apache.cassandra.io.sstable.CorruptSSTableException;
 -import org.apache.log4j.PropertyConfigurator;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
@@@ -173,9 -206,16 +175,16 @@@ public class CassandraDaemo
  if (e2 instanceof FSError)
  

git commit: Improve nodetool cleanup/scrub/upgradesstables failure handling.

2014-03-18 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 35b215136 - 17945ab0d


Improve nodetool cleanup/scrub/upgradesstables failure handling.

Patch by marcuse, reviewed by jbellis for CASSANDRA-6774


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

Branch: refs/heads/cassandra-2.1
Commit: 17945ab0d0bd65d35f99e4577f5c516aa60423f1
Parents: 35b2151
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Mar 18 09:38:14 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Tue Mar 18 09:46:26 2014 +0100

--
 CHANGES.txt |   1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  17 +--
 .../db/compaction/CompactionManager.java| 105 ++-
 .../SizeTieredCompactionStrategy.java   |   2 +-
 .../cassandra/service/StorageService.java   |  30 --
 .../cassandra/service/StorageServiceMBean.java  |   8 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |  46 ++--
 .../org/apache/cassandra/tools/NodeTool.java|   6 +-
 8 files changed, 161 insertions(+), 54 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/17945ab0/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 04e16a0..384d995 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -22,6 +22,7 @@
  * Fix stress smart Thrift client to pick servers correctly (CASSANDRA-6848)
  * Add logging levels (minimal, normal or verbose) to stress tool 
(CASSANDRA-6849)
  * Fix race condition in Batch CLE (CASSANDRA-6860)
+ * Improve cleanup/scrub/upgradesstables failure handling (CASSANDRA-6774)
 Merged from 2.0:
  * Update hadoop_cql3_word_count example (CASSANDRA-6793)
  * Fix handling of RejectedExecution in sync Thrift server (CASSANDRA-6788)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/17945ab0/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
--
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java 
b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 40632b2..e116574 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1294,22 +1294,22 @@ public class ColumnFamilyStore implements 
ColumnFamilyStoreMBean
 return maxFile;
 }
 
-public void forceCleanup() throws ExecutionException, InterruptedException
+public CompactionManager.AllSSTableOpStatus forceCleanup() throws 
ExecutionException, InterruptedException
 {
-CompactionManager.instance.performCleanup(ColumnFamilyStore.this);
+return 
CompactionManager.instance.performCleanup(ColumnFamilyStore.this);
 }
 
-public void scrub(boolean disableSnapshot, boolean skipCorrupted) throws 
ExecutionException, InterruptedException
+public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, 
boolean skipCorrupted) throws ExecutionException, InterruptedException
 {
 // skip snapshot creation during scrub, SEE JIRA 5891
 if(!disableSnapshot)
 snapshotWithoutFlush(pre-scrub- + System.currentTimeMillis());
-CompactionManager.instance.performScrub(ColumnFamilyStore.this, 
skipCorrupted);
+return CompactionManager.instance.performScrub(ColumnFamilyStore.this, 
skipCorrupted);
 }
 
-public void sstablesRewrite(boolean excludeCurrentVersion) throws 
ExecutionException, InterruptedException
+public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean 
excludeCurrentVersion) throws ExecutionException, InterruptedException
 {
-
CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, 
excludeCurrentVersion);
+return 
CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, 
excludeCurrentVersion);
 }
 
 public void markObsolete(CollectionSSTableReader sstables, OperationType 
compactionType)
@@ -2414,7 +2414,8 @@ public class ColumnFamilyStore implements 
ColumnFamilyStoreMBean
 {
 if (!cfs.getDataTracker().getCompacting().isEmpty())
 {
-logger.warn(Unable to cancel in-progress compactions 
for {}.  Probably there is an unusually large row in progress somewhere.  It is 
also possible that buggy code left some sstables compacting after it was done 
with them, metadata.cfName);
+logger.warn(Unable to cancel in-progress compactions 
for {}.  Perhaps there is an unusually large row in progress somewhere

[1/2] git commit: Improve nodetool cleanup/scrub/upgradesstables failure handling.

2014-03-18 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/trunk 6e66cf5f2 - 5b88a0f62


Improve nodetool cleanup/scrub/upgradesstables failure handling.

Patch by marcuse, reviewed by jbellis for CASSANDRA-6774


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

Branch: refs/heads/trunk
Commit: 17945ab0d0bd65d35f99e4577f5c516aa60423f1
Parents: 35b2151
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Mar 18 09:38:14 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Tue Mar 18 09:46:26 2014 +0100

--
 CHANGES.txt |   1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  17 +--
 .../db/compaction/CompactionManager.java| 105 ++-
 .../SizeTieredCompactionStrategy.java   |   2 +-
 .../cassandra/service/StorageService.java   |  30 --
 .../cassandra/service/StorageServiceMBean.java  |   8 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |  46 ++--
 .../org/apache/cassandra/tools/NodeTool.java|   6 +-
 8 files changed, 161 insertions(+), 54 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/17945ab0/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 04e16a0..384d995 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -22,6 +22,7 @@
  * Fix stress smart Thrift client to pick servers correctly (CASSANDRA-6848)
  * Add logging levels (minimal, normal or verbose) to stress tool 
(CASSANDRA-6849)
  * Fix race condition in Batch CLE (CASSANDRA-6860)
+ * Improve cleanup/scrub/upgradesstables failure handling (CASSANDRA-6774)
 Merged from 2.0:
  * Update hadoop_cql3_word_count example (CASSANDRA-6793)
  * Fix handling of RejectedExecution in sync Thrift server (CASSANDRA-6788)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/17945ab0/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
--
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java 
b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 40632b2..e116574 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1294,22 +1294,22 @@ public class ColumnFamilyStore implements 
ColumnFamilyStoreMBean
 return maxFile;
 }
 
-public void forceCleanup() throws ExecutionException, InterruptedException
+public CompactionManager.AllSSTableOpStatus forceCleanup() throws 
ExecutionException, InterruptedException
 {
-CompactionManager.instance.performCleanup(ColumnFamilyStore.this);
+return 
CompactionManager.instance.performCleanup(ColumnFamilyStore.this);
 }
 
-public void scrub(boolean disableSnapshot, boolean skipCorrupted) throws 
ExecutionException, InterruptedException
+public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, 
boolean skipCorrupted) throws ExecutionException, InterruptedException
 {
 // skip snapshot creation during scrub, SEE JIRA 5891
 if(!disableSnapshot)
 snapshotWithoutFlush(pre-scrub- + System.currentTimeMillis());
-CompactionManager.instance.performScrub(ColumnFamilyStore.this, 
skipCorrupted);
+return CompactionManager.instance.performScrub(ColumnFamilyStore.this, 
skipCorrupted);
 }
 
-public void sstablesRewrite(boolean excludeCurrentVersion) throws 
ExecutionException, InterruptedException
+public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean 
excludeCurrentVersion) throws ExecutionException, InterruptedException
 {
-
CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, 
excludeCurrentVersion);
+return 
CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, 
excludeCurrentVersion);
 }
 
 public void markObsolete(CollectionSSTableReader sstables, OperationType 
compactionType)
@@ -2414,7 +2414,8 @@ public class ColumnFamilyStore implements 
ColumnFamilyStoreMBean
 {
 if (!cfs.getDataTracker().getCompacting().isEmpty())
 {
-logger.warn(Unable to cancel in-progress compactions 
for {}.  Probably there is an unusually large row in progress somewhere.  It is 
also possible that buggy code left some sstables compacting after it was done 
with them, metadata.cfName);
+logger.warn(Unable to cancel in-progress compactions 
for {}.  Perhaps there is an unusually large row in progress somewhere, or the 
system

[2/2] git commit: Merge branch 'cassandra-2.1' into trunk

2014-03-18 Thread marcuse
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 5b88a0f6281668bad154e5c91accdab34d10dddb
Parents: 6e66cf5 17945ab
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Mar 18 09:47:50 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Tue Mar 18 09:47:50 2014 +0100

--
 CHANGES.txt |   1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  17 +--
 .../db/compaction/CompactionManager.java| 105 ++-
 .../SizeTieredCompactionStrategy.java   |   2 +-
 .../cassandra/service/StorageService.java   |  30 --
 .../cassandra/service/StorageServiceMBean.java  |   8 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |  46 ++--
 .../org/apache/cassandra/tools/NodeTool.java|   6 +-
 8 files changed, 161 insertions(+), 54 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b88a0f6/CHANGES.txt
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b88a0f6/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
--



[1/4] ByteBuffer write() methods for serializing sstables

2014-03-19 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 2a2141ce6 - 75508ec89


http://git-wip-us.apache.org/repos/asf/cassandra/blob/75508ec8/test/unit/org/apache/cassandra/utils/FastByteOperationsTest.java
--
diff --git a/test/unit/org/apache/cassandra/utils/FastByteOperationsTest.java 
b/test/unit/org/apache/cassandra/utils/FastByteOperationsTest.java
new file mode 100644
index 000..4f6d652
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/FastByteOperationsTest.java
@@ -0,0 +1,162 @@
+/*
+ * 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.utils;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Random;
+
+import org.junit.Test;
+
+public class FastByteOperationsTest
+{
+
+private static final FastByteOperations.PureJavaOperations PJO = new 
FastByteOperations.PureJavaOperations();
+private static final FastByteOperations.UnsafeOperations UO = new 
FastByteOperations.UnsafeOperations();
+private static final Random rand = new Random(0);
+private static final ByteBuffer dbuf1 = ByteBuffer.allocateDirect(150);
+private static final ByteBuffer dbuf2 = ByteBuffer.allocateDirect(150);
+private static final ByteBuffer hbuf1 = ByteBuffer.allocate(150);
+private static final ByteBuffer hbuf2 = ByteBuffer.allocate(150);
+
+@Test
+public void testFastByteCopy()
+{
+byte[] bytes1 = new byte[128];
+byte[] empty = new byte[128];
+rand.nextBytes(bytes1);
+testCopy(bytes1, wrap1(bytes1, true), wrap2(empty, true), PJO);
+testCopy(bytes1, wrap1(bytes1, true), wrap2(empty, false), PJO);
+testCopy(bytes1, wrap1(bytes1, false), wrap2(empty, true), PJO);
+testCopy(bytes1, wrap1(bytes1, false), wrap2(empty, false), PJO);
+testCopy(bytes1, wrap1(bytes1, true), wrap2(empty, true), UO);
+testCopy(bytes1, wrap1(bytes1, true), wrap2(empty, false), UO);
+testCopy(bytes1, wrap1(bytes1, false), wrap2(empty, true), UO);
+testCopy(bytes1, wrap1(bytes1, false), wrap2(empty, false), UO);
+}
+
+private void testCopy(byte[] canon, ByteBuffer src, ByteBuffer trg, 
FastByteOperations.ByteOperations ops)
+{
+byte[] result = new byte[src.remaining()];
+ops.copy(src, src.position(), trg, trg.position(), src.remaining());
+ops.copy(trg, trg.position(), result, 0, trg.remaining());
+assert firstdiff(canon, result)  0;
+}
+
+private static int firstdiff(byte[] canon, byte[] test)
+{
+for (int i = 0 ; i  canon.length ; i++)
+if (canon[i] != test[i])
+return i;
+return -1;
+}
+
+@Test
+public void testFastByteComparisons()
+{
+byte[] bytes1 = new byte[128];
+for (int i = 0 ; i  1000 ; i++)
+{
+rand.nextBytes(bytes1);
+for (int j = 0 ; j  16 ; j++)
+{
+byte[] bytes2 = Arrays.copyOf(bytes1, bytes1.length - j);
+testTwiddleOneByteComparisons(bytes1, bytes2, 16, true, 1);
+testTwiddleOneByteComparisons(bytes1, bytes2, 16, true, -1);
+testTwiddleOneByteComparisons(bytes1, bytes2, 16, false, 1);
+testTwiddleOneByteComparisons(bytes1, bytes2, 16, false, -1);
+testTwiddleOneByteComparisons(bytes1, bytes2, 16, true, 128);
+testTwiddleOneByteComparisons(bytes1, bytes2, 16, false, 128);
+}
+}
+}
+
+private void testTwiddleOneByteComparisons(byte[] bytes1, byte[] bytes2, 
int count, boolean start, int inc)
+{
+for (int j = 0 ; j  count ; j++)
+{
+int index = start ? j : bytes2.length - (j + 1);
+bytes2[index] += inc;
+testComparisons(bytes1, bytes2);
+bytes2[index] -= inc;
+}
+}
+
+private static ByteBuffer wrap1(byte[] bytes, boolean direct)
+{
+return slice(bytes, direct ? dbuf1 : hbuf1);
+}
+
+private static ByteBuffer wrap2(byte[] bytes, boolean direct)
+{
+return slice(bytes, direct ? dbuf2 : hbuf2);
+}
+
+

[5/5] git commit: Merge branch 'cassandra-2.1' into trunk

2014-03-19 Thread marcuse
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 9f057832e3796635eb4fc1f6d757dffed0d47c4e
Parents: 00317fc 75508ec
Author: Marcus Eriksson marc...@apache.org
Authored: Wed Mar 19 18:01:48 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Wed Mar 19 18:01:48 2014 +0100

--
 CHANGES.txt |   1 +
 .../apache/cassandra/cache/AutoSavingCache.java |   3 +-
 .../cache/SerializingCacheProvider.java |   4 +-
 .../apache/cassandra/db/BatchlogManager.java|  12 +-
 .../cassandra/db/ColumnFamilySerializer.java|   7 +-
 .../org/apache/cassandra/db/ColumnIndex.java|   6 +-
 .../apache/cassandra/db/ColumnSerializer.java   |   4 +-
 .../apache/cassandra/db/CounterMutation.java|   3 +-
 .../org/apache/cassandra/db/DeletionInfo.java   |   4 +-
 .../org/apache/cassandra/db/DeletionTime.java   |   4 +-
 src/java/org/apache/cassandra/db/Mutation.java  |   4 +-
 .../org/apache/cassandra/db/OnDiskAtom.java |   3 +-
 .../apache/cassandra/db/PagedRangeCommand.java  |   4 +-
 .../apache/cassandra/db/RangeSliceCommand.java  |   4 +-
 .../apache/cassandra/db/RangeSliceReply.java|   4 +-
 .../org/apache/cassandra/db/RangeTombstone.java |   5 +-
 .../apache/cassandra/db/RangeTombstoneList.java |   4 +-
 .../org/apache/cassandra/db/ReadCommand.java|   4 +-
 .../org/apache/cassandra/db/ReadResponse.java   |   5 +-
 src/java/org/apache/cassandra/db/Row.java   |   3 +-
 .../org/apache/cassandra/db/RowIndexEntry.java  |   4 +-
 .../org/apache/cassandra/db/RowPosition.java|   4 +-
 .../cassandra/db/SliceByNamesReadCommand.java   |   3 +-
 .../cassandra/db/SliceFromReadCommand.java  |   4 +-
 .../apache/cassandra/db/SnapshotCommand.java|   4 +-
 .../apache/cassandra/db/TruncateResponse.java   |   4 +-
 .../org/apache/cassandra/db/Truncation.java |   4 +-
 .../org/apache/cassandra/db/WriteResponse.java  |   4 +-
 .../cassandra/db/commitlog/CommitLog.java   |   8 +-
 .../cassandra/db/commitlog/ReplayPosition.java  |   4 +-
 .../db/compaction/AbstractCompactedRow.java |   3 +-
 .../db/compaction/LazilyCompactedRow.java   |   3 +-
 .../cassandra/db/composites/AbstractCType.java  |   4 +-
 .../db/composites/AbstractCellNameType.java |   4 +-
 .../apache/cassandra/db/filter/ColumnSlice.java |   4 +-
 .../cassandra/db/filter/IDiskAtomFilter.java|   5 +-
 .../cassandra/db/filter/NamesQueryFilter.java   |   4 +-
 .../cassandra/db/filter/SliceQueryFilter.java   |   4 +-
 .../apache/cassandra/dht/AbstractBounds.java|   3 +-
 .../org/apache/cassandra/dht/BootStrapper.java  |   4 +-
 src/java/org/apache/cassandra/dht/Token.java|   4 +-
 .../org/apache/cassandra/gms/EchoMessage.java   |   4 +-
 .../org/apache/cassandra/gms/EndpointState.java |   3 +-
 .../org/apache/cassandra/gms/GossipDigest.java  |   3 +-
 .../apache/cassandra/gms/GossipDigestAck.java   |   4 +-
 .../apache/cassandra/gms/GossipDigestAck2.java  |   3 +-
 .../apache/cassandra/gms/GossipDigestSyn.java   |   5 +-
 .../apache/cassandra/gms/HeartBeatState.java|   3 +-
 .../apache/cassandra/gms/VersionedValue.java|   3 +-
 .../apache/cassandra/io/ISSTableSerializer.java |   5 +-
 .../org/apache/cassandra/io/ISerializer.java|   6 +-
 .../cassandra/io/IVersionedSerializer.java  |   6 +-
 .../io/compress/CompressionMetadata.java|   3 +-
 .../io/compress/CompressionParameters.java  |   4 +-
 .../cassandra/io/sstable/IndexHelper.java   |   3 +-
 .../cassandra/io/sstable/IndexSummary.java  |   7 +-
 .../cassandra/io/sstable/SSTableReader.java |   4 +-
 .../cassandra/io/sstable/SSTableWriter.java |   4 +-
 .../io/sstable/metadata/CompactionMetadata.java |   4 +-
 .../metadata/IMetadataComponentSerializer.java  |   5 +-
 .../sstable/metadata/IMetadataSerializer.java   |   6 +-
 .../metadata/LegacyMetadataSerializer.java  |   3 +-
 .../io/sstable/metadata/MetadataSerializer.java |   6 +-
 .../io/sstable/metadata/StatsMetadata.java  |   4 +-
 .../io/sstable/metadata/ValidationMetadata.java |   4 +-
 .../cassandra/io/util/AbstractDataOutput.java   |  43 +-
 .../io/util/ByteBufferOutputStream.java |  46 --
 .../io/util/ChecksummedOutputStream.java|  56 ---
 .../cassandra/io/util/DataOutputBuffer.java |  23 +-
 .../cassandra/io/util/DataOutputByteBuffer.java |  59 +++
 .../cassandra/io/util/DataOutputPlus.java   |  32 ++
 .../io/util/DataOutputStreamAndChannel.java |  55 +++
 .../cassandra/io/util/DataOutputStreamPlus.java |  57 +++
 .../io/util/FastByteArrayOutputStream.java  |  11 +
 

[1/5] ByteBuffer write() methods for serializing sstables

2014-03-19 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/trunk 00317fc8f - 9f057832e


http://git-wip-us.apache.org/repos/asf/cassandra/blob/75508ec8/test/unit/org/apache/cassandra/utils/FastByteOperationsTest.java
--
diff --git a/test/unit/org/apache/cassandra/utils/FastByteOperationsTest.java 
b/test/unit/org/apache/cassandra/utils/FastByteOperationsTest.java
new file mode 100644
index 000..4f6d652
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/FastByteOperationsTest.java
@@ -0,0 +1,162 @@
+/*
+ * 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.utils;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Random;
+
+import org.junit.Test;
+
+public class FastByteOperationsTest
+{
+
+private static final FastByteOperations.PureJavaOperations PJO = new 
FastByteOperations.PureJavaOperations();
+private static final FastByteOperations.UnsafeOperations UO = new 
FastByteOperations.UnsafeOperations();
+private static final Random rand = new Random(0);
+private static final ByteBuffer dbuf1 = ByteBuffer.allocateDirect(150);
+private static final ByteBuffer dbuf2 = ByteBuffer.allocateDirect(150);
+private static final ByteBuffer hbuf1 = ByteBuffer.allocate(150);
+private static final ByteBuffer hbuf2 = ByteBuffer.allocate(150);
+
+@Test
+public void testFastByteCopy()
+{
+byte[] bytes1 = new byte[128];
+byte[] empty = new byte[128];
+rand.nextBytes(bytes1);
+testCopy(bytes1, wrap1(bytes1, true), wrap2(empty, true), PJO);
+testCopy(bytes1, wrap1(bytes1, true), wrap2(empty, false), PJO);
+testCopy(bytes1, wrap1(bytes1, false), wrap2(empty, true), PJO);
+testCopy(bytes1, wrap1(bytes1, false), wrap2(empty, false), PJO);
+testCopy(bytes1, wrap1(bytes1, true), wrap2(empty, true), UO);
+testCopy(bytes1, wrap1(bytes1, true), wrap2(empty, false), UO);
+testCopy(bytes1, wrap1(bytes1, false), wrap2(empty, true), UO);
+testCopy(bytes1, wrap1(bytes1, false), wrap2(empty, false), UO);
+}
+
+private void testCopy(byte[] canon, ByteBuffer src, ByteBuffer trg, 
FastByteOperations.ByteOperations ops)
+{
+byte[] result = new byte[src.remaining()];
+ops.copy(src, src.position(), trg, trg.position(), src.remaining());
+ops.copy(trg, trg.position(), result, 0, trg.remaining());
+assert firstdiff(canon, result)  0;
+}
+
+private static int firstdiff(byte[] canon, byte[] test)
+{
+for (int i = 0 ; i  canon.length ; i++)
+if (canon[i] != test[i])
+return i;
+return -1;
+}
+
+@Test
+public void testFastByteComparisons()
+{
+byte[] bytes1 = new byte[128];
+for (int i = 0 ; i  1000 ; i++)
+{
+rand.nextBytes(bytes1);
+for (int j = 0 ; j  16 ; j++)
+{
+byte[] bytes2 = Arrays.copyOf(bytes1, bytes1.length - j);
+testTwiddleOneByteComparisons(bytes1, bytes2, 16, true, 1);
+testTwiddleOneByteComparisons(bytes1, bytes2, 16, true, -1);
+testTwiddleOneByteComparisons(bytes1, bytes2, 16, false, 1);
+testTwiddleOneByteComparisons(bytes1, bytes2, 16, false, -1);
+testTwiddleOneByteComparisons(bytes1, bytes2, 16, true, 128);
+testTwiddleOneByteComparisons(bytes1, bytes2, 16, false, 128);
+}
+}
+}
+
+private void testTwiddleOneByteComparisons(byte[] bytes1, byte[] bytes2, 
int count, boolean start, int inc)
+{
+for (int j = 0 ; j  count ; j++)
+{
+int index = start ? j : bytes2.length - (j + 1);
+bytes2[index] += inc;
+testComparisons(bytes1, bytes2);
+bytes2[index] -= inc;
+}
+}
+
+private static ByteBuffer wrap1(byte[] bytes, boolean direct)
+{
+return slice(bytes, direct ? dbuf1 : hbuf1);
+}
+
+private static ByteBuffer wrap2(byte[] bytes, boolean direct)
+{
+return slice(bytes, direct ? dbuf2 : hbuf2);
+}
+
+private 

git commit: Make OpOrder AutoCloseable

2014-03-21 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 53e22120d - 269f81052


Make OpOrder AutoCloseable

Patch by benedict, reviewed by marcuse for CASSANDRA-6901.


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

Branch: refs/heads/cassandra-2.1
Commit: 269f81052e42d36f9a3bee684464543b7074b6b9
Parents: 53e2212
Author: belliottsmith git...@sub.laerad.com
Authored: Fri Mar 21 15:04:36 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Fri Mar 21 15:05:39 2014 +0100

--
 CHANGES.txt |  1 +
 src/java/org/apache/cassandra/db/Keyspace.java  | 27 --
 .../db/commitlog/CommitLogSegment.java  | 13 ++---
 .../db/compaction/CompactionManager.java| 54 ++--
 .../db/index/SecondaryIndexManager.java | 34 
 .../db/index/composites/CompositesIndex.java|  7 +--
 .../cassandra/db/index/keys/KeysSearcher.java   |  9 +---
 .../cassandra/utils/concurrent/OpOrder.java | 21 +++-
 .../cassandra/concurrent/LongOpOrderTest.java   | 24 -
 9 files changed, 99 insertions(+), 91 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/269f8105/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 7e2ed4d..2949b6a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -27,6 +27,7 @@
  * Proper compare function for CollectionType (CASSANDRA-6783)
  * Update native server to Netty 4 (CASSANDRA-6236)
  * Fix off-by-one error in stress (CASSANDRA-6883)
+ * Make OpOrder AutoCloseable (CASSANDRA-6901)
 Merged from 2.0:
  * Add uuid() function (CASSANDRA-6473)
  * Omit tombstones from schema digests (CASSANDRA-6862)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/269f8105/src/java/org/apache/cassandra/db/Keyspace.java
--
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java 
b/src/java/org/apache/cassandra/db/Keyspace.java
index 436aca0..fabd433 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -19,16 +19,19 @@ package org.apache.cassandra.db;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.Future;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
-
-import org.apache.cassandra.utils.concurrent.OpOrder;
-import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -37,6 +40,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
@@ -45,6 +49,7 @@ import 
org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.service.pager.QueryPagers;
 import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.concurrent.OpOrder;
 
 /**
  * It represents a Keyspace.
@@ -338,8 +343,7 @@ public class Keyspace
  */
 public void apply(Mutation mutation, boolean writeCommitLog, boolean 
updateIndexes)
 {
-final OpOrder.Group opGroup = writeOrder.start();
-try
+try (OpOrder.Group opGroup = writeOrder.start())
 {
 // write the mutation to the commitlog and memtables
 final ReplayPosition replayPosition;
@@ -370,10 +374,6 @@ public class Keyspace
 cfs.apply(key, cf, updater, opGroup, replayPosition);
 }
 }
-finally
-{
-opGroup.finishOne();
-}
 }
 
 public AbstractReplicationStrategy getReplicationStrategy()
@@ -391,8 +391,7 @@ public class Keyspace
 if (logger.isDebugEnabled())
 logger.debug(Indexing row {} , 
cfs.metadata.getKeyValidator().getString(key.key));
 
-final OpOrder.Group opGroup = cfs.keyspace.writeOrder.start();
-try
+try (OpOrder.Group opGroup

[1/2] git commit: Make OpOrder AutoCloseable

2014-03-21 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/trunk 74290df23 - 0b68b6dd7


Make OpOrder AutoCloseable

Patch by benedict, reviewed by marcuse for CASSANDRA-6901.


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

Branch: refs/heads/trunk
Commit: 269f81052e42d36f9a3bee684464543b7074b6b9
Parents: 53e2212
Author: belliottsmith git...@sub.laerad.com
Authored: Fri Mar 21 15:04:36 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Fri Mar 21 15:05:39 2014 +0100

--
 CHANGES.txt |  1 +
 src/java/org/apache/cassandra/db/Keyspace.java  | 27 --
 .../db/commitlog/CommitLogSegment.java  | 13 ++---
 .../db/compaction/CompactionManager.java| 54 ++--
 .../db/index/SecondaryIndexManager.java | 34 
 .../db/index/composites/CompositesIndex.java|  7 +--
 .../cassandra/db/index/keys/KeysSearcher.java   |  9 +---
 .../cassandra/utils/concurrent/OpOrder.java | 21 +++-
 .../cassandra/concurrent/LongOpOrderTest.java   | 24 -
 9 files changed, 99 insertions(+), 91 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/269f8105/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 7e2ed4d..2949b6a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -27,6 +27,7 @@
  * Proper compare function for CollectionType (CASSANDRA-6783)
  * Update native server to Netty 4 (CASSANDRA-6236)
  * Fix off-by-one error in stress (CASSANDRA-6883)
+ * Make OpOrder AutoCloseable (CASSANDRA-6901)
 Merged from 2.0:
  * Add uuid() function (CASSANDRA-6473)
  * Omit tombstones from schema digests (CASSANDRA-6862)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/269f8105/src/java/org/apache/cassandra/db/Keyspace.java
--
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java 
b/src/java/org/apache/cassandra/db/Keyspace.java
index 436aca0..fabd433 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -19,16 +19,19 @@ package org.apache.cassandra.db;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.Future;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
-
-import org.apache.cassandra.utils.concurrent.OpOrder;
-import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -37,6 +40,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
@@ -45,6 +49,7 @@ import 
org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.service.pager.QueryPagers;
 import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.concurrent.OpOrder;
 
 /**
  * It represents a Keyspace.
@@ -338,8 +343,7 @@ public class Keyspace
  */
 public void apply(Mutation mutation, boolean writeCommitLog, boolean 
updateIndexes)
 {
-final OpOrder.Group opGroup = writeOrder.start();
-try
+try (OpOrder.Group opGroup = writeOrder.start())
 {
 // write the mutation to the commitlog and memtables
 final ReplayPosition replayPosition;
@@ -370,10 +374,6 @@ public class Keyspace
 cfs.apply(key, cf, updater, opGroup, replayPosition);
 }
 }
-finally
-{
-opGroup.finishOne();
-}
 }
 
 public AbstractReplicationStrategy getReplicationStrategy()
@@ -391,8 +391,7 @@ public class Keyspace
 if (logger.isDebugEnabled())
 logger.debug(Indexing row {} , 
cfs.metadata.getKeyValidator().getString(key.key));
 
-final OpOrder.Group opGroup = cfs.keyspace.writeOrder.start();
-try
+try (OpOrder.Group opGroup

[2/2] git commit: Merge branch 'cassandra-2.1' into trunk

2014-03-21 Thread marcuse
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 0b68b6dd7edf7627672d93ad8d78ca6434e8c617
Parents: 74290df 269f810
Author: Marcus Eriksson marc...@apache.org
Authored: Fri Mar 21 15:06:03 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Fri Mar 21 15:06:03 2014 +0100

--
 CHANGES.txt |  1 +
 src/java/org/apache/cassandra/db/Keyspace.java  | 27 --
 .../db/commitlog/CommitLogSegment.java  | 13 ++---
 .../db/compaction/CompactionManager.java| 54 ++--
 .../db/index/SecondaryIndexManager.java | 34 
 .../db/index/composites/CompositesIndex.java|  7 +--
 .../cassandra/db/index/keys/KeysSearcher.java   |  9 +---
 .../cassandra/utils/concurrent/OpOrder.java | 21 +++-
 .../cassandra/concurrent/LongOpOrderTest.java   | 24 -
 9 files changed, 99 insertions(+), 91 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0b68b6dd/CHANGES.txt
--



git commit: Remove avro usage in DataOutputTest

2014-03-24 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 fbc112d4b - 5baa72f7f


Remove avro usage in DataOutputTest


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

Branch: refs/heads/cassandra-2.1
Commit: 5baa72f7f299b4ec190ddb30b897d5519a6a2a75
Parents: fbc112d
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 24 12:36:05 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 24 12:36:05 2014 +0100

--
 test/unit/org/apache/cassandra/io/util/DataOutputTest.java | 6 ++
 1 file changed, 2 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5baa72f7/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
--
diff --git a/test/unit/org/apache/cassandra/io/util/DataOutputTest.java 
b/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
index 4eeec4d..2a8c7a9 100644
--- a/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
+++ b/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
@@ -31,14 +31,12 @@ import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.channels.Channels;
-import java.util.Arrays;
 import java.util.Random;
 import java.util.concurrent.ThreadLocalRandom;
 
 import org.junit.Assert;
 import org.junit.Test;
 
-import org.apache.avro.util.ByteBufferInputStream;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class DataOutputTest
@@ -79,7 +77,7 @@ public class DataOutputTest
 ByteBuffer buf = wrap(new byte[345], true);
 DataOutputByteBuffer write = new DataOutputByteBuffer(buf.duplicate());
 DataInput canon = testWrite(write);
-DataInput test = new DataInputStream(new 
ByteBufferInputStream(Arrays.asList(buf)));
+DataInput test = new DataInputStream(new 
ByteArrayInputStream(ByteBufferUtil.getArray(buf)));
 testRead(test, canon);
 }
 
@@ -89,7 +87,7 @@ public class DataOutputTest
 ByteBuffer buf = wrap(new byte[345], false);
 DataOutputByteBuffer write = new DataOutputByteBuffer(buf.duplicate());
 DataInput canon = testWrite(write);
-DataInput test = new DataInputStream(new 
ByteBufferInputStream(Arrays.asList(buf)));
+DataInput test = new DataInputStream(new 
ByteArrayInputStream(ByteBufferUtil.getArray(buf)));
 testRead(test, canon);
 }
 



[1/2] git commit: Remove avro usage in DataOutputTest

2014-03-24 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/trunk 6f24097f6 - 6838790f8


Remove avro usage in DataOutputTest


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

Branch: refs/heads/trunk
Commit: 5baa72f7f299b4ec190ddb30b897d5519a6a2a75
Parents: fbc112d
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 24 12:36:05 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 24 12:36:05 2014 +0100

--
 test/unit/org/apache/cassandra/io/util/DataOutputTest.java | 6 ++
 1 file changed, 2 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5baa72f7/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
--
diff --git a/test/unit/org/apache/cassandra/io/util/DataOutputTest.java 
b/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
index 4eeec4d..2a8c7a9 100644
--- a/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
+++ b/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
@@ -31,14 +31,12 @@ import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.channels.Channels;
-import java.util.Arrays;
 import java.util.Random;
 import java.util.concurrent.ThreadLocalRandom;
 
 import org.junit.Assert;
 import org.junit.Test;
 
-import org.apache.avro.util.ByteBufferInputStream;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class DataOutputTest
@@ -79,7 +77,7 @@ public class DataOutputTest
 ByteBuffer buf = wrap(new byte[345], true);
 DataOutputByteBuffer write = new DataOutputByteBuffer(buf.duplicate());
 DataInput canon = testWrite(write);
-DataInput test = new DataInputStream(new 
ByteBufferInputStream(Arrays.asList(buf)));
+DataInput test = new DataInputStream(new 
ByteArrayInputStream(ByteBufferUtil.getArray(buf)));
 testRead(test, canon);
 }
 
@@ -89,7 +87,7 @@ public class DataOutputTest
 ByteBuffer buf = wrap(new byte[345], false);
 DataOutputByteBuffer write = new DataOutputByteBuffer(buf.duplicate());
 DataInput canon = testWrite(write);
-DataInput test = new DataInputStream(new 
ByteBufferInputStream(Arrays.asList(buf)));
+DataInput test = new DataInputStream(new 
ByteArrayInputStream(ByteBufferUtil.getArray(buf)));
 testRead(test, canon);
 }
 



[2/2] git commit: Merge branch 'cassandra-2.1' into trunk

2014-03-24 Thread marcuse
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 6838790f869dfcd773c83fd0165c699e6984540d
Parents: 6f24097 5baa72f
Author: Marcus Eriksson marc...@apache.org
Authored: Mon Mar 24 12:36:20 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Mon Mar 24 12:36:20 2014 +0100

--
 test/unit/org/apache/cassandra/io/util/DataOutputTest.java | 6 ++
 1 file changed, 2 insertions(+), 4 deletions(-)
--




git commit: Make sure we keep the sstable level when running upgradesstables.

2014-03-31 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.0 0fce7ace0 - a90b98e90


Make sure we keep the sstable level when running upgradesstables.

Patch by marcuse; reviewed by yukim for CASSANDRA-6958


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

Branch: refs/heads/cassandra-2.0
Commit: a90b98e902fc54f62a379a6ce1e6ddead43ba1ec
Parents: 0fce7ac
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Apr 1 07:37:12 2014 +0200
Committer: Marcus Eriksson marc...@apache.org
Committed: Tue Apr 1 07:40:35 2014 +0200

--
 CHANGES.txt |  1 +
 .../db/compaction/AbstractCompactionStrategy.java   |  2 ++
 .../cassandra/db/compaction/CompactionManager.java  |  2 +-
 .../db/compaction/LeveledCompactionStrategy.java| 16 
 .../db/compaction/SizeTieredCompactionStrategy.java |  6 ++
 .../apache/cassandra/db/compaction/Upgrader.java|  2 ++
 6 files changed, 28 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a90b98e9/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index b4f84e4..3326c6c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -36,6 +36,7 @@
  * Add CqlRecordReader using native pagination (CASSANDRA-6311)
  * Add QueryHandler interface (CASSANDRA-6659)
  * Track liveRatio per-memtable, not per-CF (CASSANDRA-6945)
+ * Make sure upgradesstables keeps sstable level (CASSANDRA-6958)
 Merged from 1.2:
  * Add UNLOGGED, COUNTER options to BATCH documentation (CASSANDRA-6816)
  * add extra SSL cipher suites (CASSANDRA-6613)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a90b98e9/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
--
diff --git 
a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java 
b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
index 5425683..d3339d5 100644
--- 
a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
+++ 
b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
@@ -167,6 +167,8 @@ public abstract class AbstractCompactionStrategy
  */
 public abstract AbstractCompactionTask 
getUserDefinedTask(CollectionSSTableReader sstables, final int gcBefore);
 
+public abstract AbstractCompactionTask 
getCompactionTask(CollectionSSTableReader sstables, final int gcBefore, long 
maxSSTableBytes);
+
 /**
  * @return the number of background tasks estimated to still be needed for 
this columnfamilystore
  */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a90b98e9/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
--
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java 
b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 453176e..31b7d8e 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -251,7 +251,7 @@ public class CompactionManager implements 
CompactionManagerMBean
 
 // SSTables are marked by the caller
 // NOTE: it is important that the task create one and only 
one sstable, even for Leveled compaction (see LeveledManifest.replace())
-CompactionTask task = new CompactionTask(cfs, 
Collections.singletonList(sstable), NO_GC);
+AbstractCompactionTask task = 
cfs.getCompactionStrategy().getCompactionTask(Collections.singleton(sstable), 
NO_GC, Long.MAX_VALUE);
 task.setUserDefined(true);
 task.setCompactionType(OperationType.UPGRADE_SSTABLES);
 task.execute(metrics);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a90b98e9/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
--
diff --git 
a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java 
b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
index 18b9fbd..8637a2e 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
@@ -152,6 +152,22 @@ public class LeveledCompactionStrategy extends 
AbstractCompactionStrategy implem
 throw new

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

2014-03-31 Thread marcuse
Merge branch 'cassandra-2.0' into cassandra-2.1

Conflicts:
CHANGES.txt


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

Branch: refs/heads/cassandra-2.1
Commit: 26191ca6968a21bf4cc1dd0a2dd1e5eef132e47c
Parents: e5639f4 a90b98e
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Apr 1 07:43:13 2014 +0200
Committer: Marcus Eriksson marc...@apache.org
Committed: Tue Apr 1 07:43:13 2014 +0200

--
 CHANGES.txt |  2 ++
 .../db/compaction/AbstractCompactionStrategy.java   |  2 ++
 .../cassandra/db/compaction/CompactionManager.java  |  2 +-
 .../db/compaction/LeveledCompactionStrategy.java| 16 
 .../db/compaction/SizeTieredCompactionStrategy.java |  6 ++
 .../apache/cassandra/db/compaction/Upgrader.java|  2 ++
 6 files changed, 29 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/26191ca6/CHANGES.txt
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26191ca6/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26191ca6/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26191ca6/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26191ca6/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26191ca6/src/java/org/apache/cassandra/db/compaction/Upgrader.java
--
diff --cc src/java/org/apache/cassandra/db/compaction/Upgrader.java
index dfadfcd,98a55e9..740a3eb
--- a/src/java/org/apache/cassandra/db/compaction/Upgrader.java
+++ b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
@@@ -75,9 -75,10 +76,10 @@@ public class Upgrade
  if (new 
File(sstable.descriptor.withGeneration(i).filenameFor(Component.DATA)).exists())
  sstableMetadataCollector.addAncestor(i);
  }
+ sstableMetadataCollector.sstableLevel(sstable.getSSTableLevel());
  }
  
 -return new SSTableWriter(cfs.getTempSSTablePath(directory), 
estimatedRows, cfs.metadata, cfs.partitioner, sstableMetadataCollector);
 +return new SSTableWriter(cfs.getTempSSTablePath(directory), 
estimatedRows, repairedAt, cfs.metadata, cfs.partitioner, 
sstableMetadataCollector);
  }
  
  public void upgrade()



[1/2] git commit: Make sure we keep the sstable level when running upgradesstables.

2014-03-31 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 e5639f44c - 26191ca69


Make sure we keep the sstable level when running upgradesstables.

Patch by marcuse; reviewed by yukim for CASSANDRA-6958


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

Branch: refs/heads/cassandra-2.1
Commit: a90b98e902fc54f62a379a6ce1e6ddead43ba1ec
Parents: 0fce7ac
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Apr 1 07:37:12 2014 +0200
Committer: Marcus Eriksson marc...@apache.org
Committed: Tue Apr 1 07:40:35 2014 +0200

--
 CHANGES.txt |  1 +
 .../db/compaction/AbstractCompactionStrategy.java   |  2 ++
 .../cassandra/db/compaction/CompactionManager.java  |  2 +-
 .../db/compaction/LeveledCompactionStrategy.java| 16 
 .../db/compaction/SizeTieredCompactionStrategy.java |  6 ++
 .../apache/cassandra/db/compaction/Upgrader.java|  2 ++
 6 files changed, 28 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a90b98e9/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index b4f84e4..3326c6c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -36,6 +36,7 @@
  * Add CqlRecordReader using native pagination (CASSANDRA-6311)
  * Add QueryHandler interface (CASSANDRA-6659)
  * Track liveRatio per-memtable, not per-CF (CASSANDRA-6945)
+ * Make sure upgradesstables keeps sstable level (CASSANDRA-6958)
 Merged from 1.2:
  * Add UNLOGGED, COUNTER options to BATCH documentation (CASSANDRA-6816)
  * add extra SSL cipher suites (CASSANDRA-6613)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a90b98e9/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
--
diff --git 
a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java 
b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
index 5425683..d3339d5 100644
--- 
a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
+++ 
b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
@@ -167,6 +167,8 @@ public abstract class AbstractCompactionStrategy
  */
 public abstract AbstractCompactionTask 
getUserDefinedTask(CollectionSSTableReader sstables, final int gcBefore);
 
+public abstract AbstractCompactionTask 
getCompactionTask(CollectionSSTableReader sstables, final int gcBefore, long 
maxSSTableBytes);
+
 /**
  * @return the number of background tasks estimated to still be needed for 
this columnfamilystore
  */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a90b98e9/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
--
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java 
b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 453176e..31b7d8e 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -251,7 +251,7 @@ public class CompactionManager implements 
CompactionManagerMBean
 
 // SSTables are marked by the caller
 // NOTE: it is important that the task create one and only 
one sstable, even for Leveled compaction (see LeveledManifest.replace())
-CompactionTask task = new CompactionTask(cfs, 
Collections.singletonList(sstable), NO_GC);
+AbstractCompactionTask task = 
cfs.getCompactionStrategy().getCompactionTask(Collections.singleton(sstable), 
NO_GC, Long.MAX_VALUE);
 task.setUserDefined(true);
 task.setCompactionType(OperationType.UPGRADE_SSTABLES);
 task.execute(metrics);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a90b98e9/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
--
diff --git 
a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java 
b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
index 18b9fbd..8637a2e 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
@@ -152,6 +152,22 @@ public class LeveledCompactionStrategy extends 
AbstractCompactionStrategy implem
 throw new

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

2014-03-31 Thread marcuse
Merge branch 'cassandra-2.0' into cassandra-2.1

Conflicts:
CHANGES.txt


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

Branch: refs/heads/trunk
Commit: 26191ca6968a21bf4cc1dd0a2dd1e5eef132e47c
Parents: e5639f4 a90b98e
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Apr 1 07:43:13 2014 +0200
Committer: Marcus Eriksson marc...@apache.org
Committed: Tue Apr 1 07:43:13 2014 +0200

--
 CHANGES.txt |  2 ++
 .../db/compaction/AbstractCompactionStrategy.java   |  2 ++
 .../cassandra/db/compaction/CompactionManager.java  |  2 +-
 .../db/compaction/LeveledCompactionStrategy.java| 16 
 .../db/compaction/SizeTieredCompactionStrategy.java |  6 ++
 .../apache/cassandra/db/compaction/Upgrader.java|  2 ++
 6 files changed, 29 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/26191ca6/CHANGES.txt
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26191ca6/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26191ca6/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26191ca6/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26191ca6/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26191ca6/src/java/org/apache/cassandra/db/compaction/Upgrader.java
--
diff --cc src/java/org/apache/cassandra/db/compaction/Upgrader.java
index dfadfcd,98a55e9..740a3eb
--- a/src/java/org/apache/cassandra/db/compaction/Upgrader.java
+++ b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
@@@ -75,9 -75,10 +76,10 @@@ public class Upgrade
  if (new 
File(sstable.descriptor.withGeneration(i).filenameFor(Component.DATA)).exists())
  sstableMetadataCollector.addAncestor(i);
  }
+ sstableMetadataCollector.sstableLevel(sstable.getSSTableLevel());
  }
  
 -return new SSTableWriter(cfs.getTempSSTablePath(directory), 
estimatedRows, cfs.metadata, cfs.partitioner, sstableMetadataCollector);
 +return new SSTableWriter(cfs.getTempSSTablePath(directory), 
estimatedRows, repairedAt, cfs.metadata, cfs.partitioner, 
sstableMetadataCollector);
  }
  
  public void upgrade()



[3/3] git commit: Merge branch 'cassandra-2.1' into trunk

2014-03-31 Thread marcuse
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 5cdfa69ca2c353247cbb9a885944292d0967e14b
Parents: 3b14239 26191ca
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Apr 1 07:44:55 2014 +0200
Committer: Marcus Eriksson marc...@apache.org
Committed: Tue Apr 1 07:44:55 2014 +0200

--
 CHANGES.txt |  2 ++
 .../db/compaction/AbstractCompactionStrategy.java   |  2 ++
 .../cassandra/db/compaction/CompactionManager.java  |  2 +-
 .../db/compaction/LeveledCompactionStrategy.java| 16 
 .../db/compaction/SizeTieredCompactionStrategy.java |  6 ++
 .../apache/cassandra/db/compaction/Upgrader.java|  2 ++
 6 files changed, 29 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5cdfa69c/CHANGES.txt
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5cdfa69c/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
--



[1/3] git commit: Make sure we keep the sstable level when running upgradesstables.

2014-03-31 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/trunk 3b1423945 - 5cdfa69ca


Make sure we keep the sstable level when running upgradesstables.

Patch by marcuse; reviewed by yukim for CASSANDRA-6958


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

Branch: refs/heads/trunk
Commit: a90b98e902fc54f62a379a6ce1e6ddead43ba1ec
Parents: 0fce7ac
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Apr 1 07:37:12 2014 +0200
Committer: Marcus Eriksson marc...@apache.org
Committed: Tue Apr 1 07:40:35 2014 +0200

--
 CHANGES.txt |  1 +
 .../db/compaction/AbstractCompactionStrategy.java   |  2 ++
 .../cassandra/db/compaction/CompactionManager.java  |  2 +-
 .../db/compaction/LeveledCompactionStrategy.java| 16 
 .../db/compaction/SizeTieredCompactionStrategy.java |  6 ++
 .../apache/cassandra/db/compaction/Upgrader.java|  2 ++
 6 files changed, 28 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a90b98e9/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index b4f84e4..3326c6c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -36,6 +36,7 @@
  * Add CqlRecordReader using native pagination (CASSANDRA-6311)
  * Add QueryHandler interface (CASSANDRA-6659)
  * Track liveRatio per-memtable, not per-CF (CASSANDRA-6945)
+ * Make sure upgradesstables keeps sstable level (CASSANDRA-6958)
 Merged from 1.2:
  * Add UNLOGGED, COUNTER options to BATCH documentation (CASSANDRA-6816)
  * add extra SSL cipher suites (CASSANDRA-6613)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a90b98e9/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
--
diff --git 
a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java 
b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
index 5425683..d3339d5 100644
--- 
a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
+++ 
b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
@@ -167,6 +167,8 @@ public abstract class AbstractCompactionStrategy
  */
 public abstract AbstractCompactionTask 
getUserDefinedTask(CollectionSSTableReader sstables, final int gcBefore);
 
+public abstract AbstractCompactionTask 
getCompactionTask(CollectionSSTableReader sstables, final int gcBefore, long 
maxSSTableBytes);
+
 /**
  * @return the number of background tasks estimated to still be needed for 
this columnfamilystore
  */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a90b98e9/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
--
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java 
b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 453176e..31b7d8e 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -251,7 +251,7 @@ public class CompactionManager implements 
CompactionManagerMBean
 
 // SSTables are marked by the caller
 // NOTE: it is important that the task create one and only 
one sstable, even for Leveled compaction (see LeveledManifest.replace())
-CompactionTask task = new CompactionTask(cfs, 
Collections.singletonList(sstable), NO_GC);
+AbstractCompactionTask task = 
cfs.getCompactionStrategy().getCompactionTask(Collections.singleton(sstable), 
NO_GC, Long.MAX_VALUE);
 task.setUserDefined(true);
 task.setCompactionType(OperationType.UPGRADE_SSTABLES);
 task.execute(metrics);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a90b98e9/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
--
diff --git 
a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java 
b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
index 18b9fbd..8637a2e 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
@@ -152,6 +152,22 @@ public class LeveledCompactionStrategy extends 
AbstractCompactionStrategy implem
 throw new UnsupportedOperationException

git commit: Don't break API in minor rev, followup to CASSANDRA-6958

2014-04-01 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.0 55fba6ab0 - 80b3e6cc3


Don't break API in minor rev, followup to CASSANDRA-6958


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

Branch: refs/heads/cassandra-2.0
Commit: 80b3e6cc3b1d6fdae91193563626f0823c92c528
Parents: 55fba6a
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Apr 1 22:03:30 2014 +0200
Committer: Marcus Eriksson marc...@apache.org
Committed: Tue Apr 1 22:16:10 2014 +0200

--
 .../cassandra/db/compaction/AbstractCompactionStrategy.java| 5 -
 .../cassandra/db/compaction/SizeTieredCompactionStrategy.java  | 6 --
 2 files changed, 4 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/80b3e6cc/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
--
diff --git 
a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java 
b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
index d3339d5..941d3d7 100644
--- 
a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
+++ 
b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
@@ -167,7 +167,10 @@ public abstract class AbstractCompactionStrategy
  */
 public abstract AbstractCompactionTask 
getUserDefinedTask(CollectionSSTableReader sstables, final int gcBefore);
 
-public abstract AbstractCompactionTask 
getCompactionTask(CollectionSSTableReader sstables, final int gcBefore, long 
maxSSTableBytes);
+public AbstractCompactionTask getCompactionTask(CollectionSSTableReader 
sstables, final int gcBefore, long maxSSTableBytes)
+{
+return new CompactionTask(cfs, sstables, gcBefore);
+}
 
 /**
  * @return the number of background tasks estimated to still be needed for 
this columnfamilystore

http://git-wip-us.apache.org/repos/asf/cassandra/blob/80b3e6cc/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
--
diff --git 
a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java 
b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
index fae52e2..0f94918 100644
--- 
a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
+++ 
b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
@@ -271,12 +271,6 @@ public class SizeTieredCompactionStrategy extends 
AbstractCompactionStrategy
 return new CompactionTask(cfs, sstables, 
gcBefore).setUserDefined(true);
 }
 
-@Override
-public AbstractCompactionTask getCompactionTask(CollectionSSTableReader 
sstables, int gcBefore, long maxSSTableBytes)
-{
-return new CompactionTask(cfs, sstables, gcBefore);
-}
-
 public int getEstimatedRemainingTasks()
 {
 return estimatedRemainingTasks;



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

2014-04-01 Thread marcuse
Merge branch 'cassandra-2.0' into cassandra-2.1


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

Branch: refs/heads/cassandra-2.1
Commit: 92b40fd90d9d590b738a325ee68ca80acfaa97f9
Parents: d6b5826 80b3e6c
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Apr 1 22:26:49 2014 +0200
Committer: Marcus Eriksson marc...@apache.org
Committed: Tue Apr 1 22:26:49 2014 +0200

--
 .../cassandra/db/compaction/AbstractCompactionStrategy.java| 5 -
 .../cassandra/db/compaction/SizeTieredCompactionStrategy.java  | 6 --
 2 files changed, 4 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/92b40fd9/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
--

http://git-wip-us.apache.org/repos/asf/cassandra/blob/92b40fd9/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
--



[1/2] git commit: Don't break API in minor rev, followup to CASSANDRA-6958

2014-04-01 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 d6b582614 - 92b40fd90


Don't break API in minor rev, followup to CASSANDRA-6958


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

Branch: refs/heads/cassandra-2.1
Commit: 80b3e6cc3b1d6fdae91193563626f0823c92c528
Parents: 55fba6a
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Apr 1 22:03:30 2014 +0200
Committer: Marcus Eriksson marc...@apache.org
Committed: Tue Apr 1 22:16:10 2014 +0200

--
 .../cassandra/db/compaction/AbstractCompactionStrategy.java| 5 -
 .../cassandra/db/compaction/SizeTieredCompactionStrategy.java  | 6 --
 2 files changed, 4 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/80b3e6cc/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
--
diff --git 
a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java 
b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
index d3339d5..941d3d7 100644
--- 
a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
+++ 
b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
@@ -167,7 +167,10 @@ public abstract class AbstractCompactionStrategy
  */
 public abstract AbstractCompactionTask 
getUserDefinedTask(CollectionSSTableReader sstables, final int gcBefore);
 
-public abstract AbstractCompactionTask 
getCompactionTask(CollectionSSTableReader sstables, final int gcBefore, long 
maxSSTableBytes);
+public AbstractCompactionTask getCompactionTask(CollectionSSTableReader 
sstables, final int gcBefore, long maxSSTableBytes)
+{
+return new CompactionTask(cfs, sstables, gcBefore);
+}
 
 /**
  * @return the number of background tasks estimated to still be needed for 
this columnfamilystore

http://git-wip-us.apache.org/repos/asf/cassandra/blob/80b3e6cc/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
--
diff --git 
a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java 
b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
index fae52e2..0f94918 100644
--- 
a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
+++ 
b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
@@ -271,12 +271,6 @@ public class SizeTieredCompactionStrategy extends 
AbstractCompactionStrategy
 return new CompactionTask(cfs, sstables, 
gcBefore).setUserDefined(true);
 }
 
-@Override
-public AbstractCompactionTask getCompactionTask(CollectionSSTableReader 
sstables, int gcBefore, long maxSSTableBytes)
-{
-return new CompactionTask(cfs, sstables, gcBefore);
-}
-
 public int getEstimatedRemainingTasks()
 {
 return estimatedRemainingTasks;



<    1   2   3   4   5   6   7   8   9   10   >