[jira] [Created] (CASSANDRA-2871) improve cfstats output

2011-07-08 Thread Wojciech Meler (JIRA)
improve cfstats output
--

 Key: CASSANDRA-2871
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2871
 Project: Cassandra
  Issue Type: Improvement
  Components: Tools
Affects Versions: 0.8.0
Reporter: Wojciech Meler
Assignee: Wojciech Meler
Priority: Trivial


Cfstats should be more grep-friendly - data should be presented in table - one 
keyspace/column family per line.


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2870) dynamic snitch + read repair off can cause LOCAL_QUORUM reads to return spurious UnavailableException

2011-07-08 Thread Sylvain Lebresne (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2870?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13061846#comment-13061846
 ] 

Sylvain Lebresne commented on CASSANDRA-2870:
-

+1

 dynamic snitch + read repair off can cause LOCAL_QUORUM reads to return 
 spurious UnavailableException
 -

 Key: CASSANDRA-2870
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2870
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.0
Reporter: Jonathan Ellis
Assignee: Jonathan Ellis
Priority: Minor
 Fix For: 0.7.8, 0.8.2

 Attachments: 2870.txt


 When Read Repair is off, we want to avoid doing requests to more nodes than 
 necessary to satisfy the ConsistencyLevel.  ReadCallback does this here:
 {code}
 this.endpoints = repair || resolver instanceof RowRepairResolver
? endpoints
: endpoints.subList(0, Math.min(endpoints.size(), 
 blockfor)); // min so as to not throw exception until assureSufficient is 
 called
 {code}
 You can see that it is assuming that the endpoints list is sorted in order 
 of preferred-ness for the read.
 Then the LOCAL_QUORUM code in DatacenterReadCallback checks to see if we have 
 enough nodes to do the read:
 {code}
 int localEndpoints = 0;
 for (InetAddress endpoint : endpoints)
 {
 if (localdc.equals(snitch.getDatacenter(endpoint)))
 localEndpoints++;
 }
 if (localEndpoints  blockfor)
 throw new UnavailableException();
 {code}
 So if repair is off (so we truncate our endpoints list) AND dynamic snitch 
 has decided that nodes in another DC are to be preferred over local ones, 
 we'll throw UE even if all the replicas are healthy.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Created] (CASSANDRA-2872) While dropping and recreating an index, incremental snapshotting can hang

2011-07-08 Thread Sylvain Lebresne (JIRA)
While dropping and recreating an index, incremental snapshotting can hang 
--

 Key: CASSANDRA-2872
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2872
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.4
Reporter: Sylvain Lebresne
Priority: Minor


When creating a hard link (at list with JNA), link() hang if the target of the
link already exists. In theory though, we should not hit that situation
because we use a new directory for each manual snapshot and the generation
number of the sstables should prevent this from hapenning with increment
snapshot.

However, when you drop, then recreate a secondary index, if the sstables are
deleted after the drop and before we recreate the index, the recreated index
sstables will start with a generation to 0. Thus, when we start backuping them
incrementally, it will conflict with the sstables of the previously dropped
index.

First, we should check for the target existance because calling link() to at
least avoid hanging. But then we must make sure that when we drop, then
recreate an index, we will either not name the sstables the same way or the
incremental snapshot use a different directory.


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Created] (CASSANDRA-2873) Typo in src/java/org/apache/cassandra/cli/CliClient

2011-07-08 Thread JIRA
Typo in src/java/org/apache/cassandra/cli/CliClient  
-

 Key: CASSANDRA-2873
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2873
 Project: Cassandra
  Issue Type: Bug
  Components: Tools
Affects Versions: 0.8.1
 Environment: ubuntu linux 10.4
Reporter: Michał Bartoszewski


I have read your documentation about syntax for creating column family and 
parameters that I can pass.
According to documentation i can use parameter :

 - keys_cache_save_period: Duration in seconds after which Cassandra should
  safe the keys cache. Caches are saved to saved_caches_directory as
  specified in conf/Cassandra.yaml. Default is 14400 or 4 hours. 

but then i was receiving error: No enum const class 
org.apache.cassandra.cli.CliClient$ColumnFamilyArgument.KEYS_CACHE_SAVE_PERIOD


In class mentioned in title we have:

protected enum ColumnFamilyArgument
115 {
116 COLUMN_TYPE,
117 COMPARATOR,
118 SUBCOMPARATOR,
119 COMMENT,
120 ROWS_CACHED,
121 ROW_CACHE_SAVE_PERIOD,
122 KEYS_CACHED,
123 KEY_CACHE_SAVE_PERIOD,    TYPO !
124 READ_REPAIR_CHANCE,
125 GC_GRACE,
126 COLUMN_METADATA,
127 MEMTABLE_OPERATIONS,
128 MEMTABLE_THROUGHPUT,
129 MEMTABLE_FLUSH_AFTER,
130 DEFAULT_VALIDATION_CLASS,
131 MIN_COMPACTION_THRESHOLD,
132 MAX_COMPACTION_THRESHOLD,
133 REPLICATE_ON_WRITE,
134 ROW_CACHE_PROVIDER,
135 KEY_VALIDATION_CLASS
136 } 

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




svn commit: r1144380 - in /cassandra/branches/cassandra-0.7: CHANGES.txt src/java/org/apache/cassandra/service/DatacenterReadCallback.java src/java/org/apache/cassandra/service/ReadCallback.java

2011-07-08 Thread jbellis
Author: jbellis
Date: Fri Jul  8 16:37:37 2011
New Revision: 1144380

URL: http://svn.apache.org/viewvc?rev=1144380view=rev
Log:
fix possibility of spuriousUnavailableException for LOCAL_QUORUM reads with 
dynamic snitch and read repair disabled
patch by jbellis; reviewed by slebresne for CASSANDRA-2870

Modified:
cassandra/branches/cassandra-0.7/CHANGES.txt

cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/DatacenterReadCallback.java

cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/ReadCallback.java

Modified: cassandra/branches/cassandra-0.7/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/CHANGES.txt?rev=1144380r1=1144379r2=1144380view=diff
==
--- cassandra/branches/cassandra-0.7/CHANGES.txt (original)
+++ cassandra/branches/cassandra-0.7/CHANGES.txt Fri Jul  8 16:37:37 2011
@@ -1,3 +1,8 @@
+0.7.8
+ * fix possibility of spurious UnavailableException for LOCAL_QUORUM
+   reads with dynamic snitch + read repair disabled (CASSANDRA-2870)
+
+
 0.7.7
  * adjust hinted handoff page size to avoid OOM with large columns 
(CASSANDRA-2652)

Modified: 
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/DatacenterReadCallback.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/DatacenterReadCallback.java?rev=1144380r1=1144379r2=1144380view=diff
==
--- 
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/DatacenterReadCallback.java
 (original)
+++ 
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/DatacenterReadCallback.java
 Fri Jul  8 16:37:37 2011
@@ -22,6 +22,7 @@ package org.apache.cassandra.service;
 
 
 import java.net.InetAddress;
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -48,6 +49,20 @@ public class DatacenterReadCallbackT e
 }
 
 @Override
+protected ListInetAddress preferredEndpoints(ListInetAddress endpoints)
+{
+ArrayListInetAddress preferred = new 
ArrayListInetAddress(blockfor);
+for (InetAddress endpoint : endpoints)
+{
+if (localdc.equals(snitch.getDatacenter(endpoint)))
+preferred.add(endpoint);
+if (preferred.size() == blockfor)
+break;
+}
+return preferred;
+}
+
+@Override
 protected boolean waitingFor(Message message)
 {
 return localdc.equals(snitch.getDatacenter(message.getFrom()));
@@ -88,7 +103,7 @@ public class DatacenterReadCallbackT e
 if (localdc.equals(snitch.getDatacenter(endpoint)))
 builder.append(endpoint).append(,);
 }
-builder.append(] are insufficient to satisfy LOCAL_QUORUM 
requirement of ).append(blockfor).append( live nodes);
+builder.append(] are insufficient to satisfy LOCAL_QUORUM 
requirement of ).append(blockfor).append( live nodes in 
').append(localdc).append(');
 logger.debug(builder.toString());
 }
 

Modified: 
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/ReadCallback.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/ReadCallback.java?rev=1144380r1=1144379r2=1144380view=diff
==
--- 
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/ReadCallback.java
 (original)
+++ 
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/ReadCallback.java
 Fri Jul  8 16:37:37 2011
@@ -80,13 +80,18 @@ public class ReadCallbackT implements 
 boolean repair = randomlyReadRepair();
 this.endpoints = repair || resolver instanceof RowRepairResolver
? endpoints
-   : endpoints.subList(0, Math.min(endpoints.size(), 
blockfor)); // min so as to not throw exception until assureSufficient is called
+   : preferredEndpoints(endpoints);
 
 if (logger.isDebugEnabled())
 logger.debug(String.format(Blockfor/repair is %s/%s; setting up 
requests to %s,
blockfor, repair, 
StringUtils.join(this.endpoints, ,)));
 }
-
+
+protected ListInetAddress preferredEndpoints(ListInetAddress endpoints)
+{
+return endpoints.subList(0, Math.min(endpoints.size(), blockfor)); // 
min so as to not throw exception until assureSufficient is called
+}
+
 private boolean randomlyReadRepair()
 {
 if (resolver instanceof RowDigestResolver)




[jira] [Commented] (CASSANDRA-2870) dynamic snitch + read repair off can cause LOCAL_QUORUM reads to return spurious UnavailableException

2011-07-08 Thread Hudson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2870?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062053#comment-13062053
 ] 

Hudson commented on CASSANDRA-2870:
---

Integrated in Cassandra-0.7 #526 (See 
[https://builds.apache.org/job/Cassandra-0.7/526/])
fix possibility of spuriousUnavailableException for LOCAL_QUORUM reads with 
dynamic snitch and read repair disabled
patch by jbellis; reviewed by slebresne for CASSANDRA-2870

jbellis : 
http://svn.apache.org/viewcvs.cgi/?root=Apache-SVNview=revrev=1144380
Files : 
* /cassandra/branches/cassandra-0.7/CHANGES.txt
* 
/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/ReadCallback.java
* 
/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/DatacenterReadCallback.java


 dynamic snitch + read repair off can cause LOCAL_QUORUM reads to return 
 spurious UnavailableException
 -

 Key: CASSANDRA-2870
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2870
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.0
Reporter: Jonathan Ellis
Assignee: Jonathan Ellis
Priority: Minor
 Fix For: 0.7.8, 0.8.2

 Attachments: 2870.txt


 When Read Repair is off, we want to avoid doing requests to more nodes than 
 necessary to satisfy the ConsistencyLevel.  ReadCallback does this here:
 {code}
 this.endpoints = repair || resolver instanceof RowRepairResolver
? endpoints
: endpoints.subList(0, Math.min(endpoints.size(), 
 blockfor)); // min so as to not throw exception until assureSufficient is 
 called
 {code}
 You can see that it is assuming that the endpoints list is sorted in order 
 of preferred-ness for the read.
 Then the LOCAL_QUORUM code in DatacenterReadCallback checks to see if we have 
 enough nodes to do the read:
 {code}
 int localEndpoints = 0;
 for (InetAddress endpoint : endpoints)
 {
 if (localdc.equals(snitch.getDatacenter(endpoint)))
 localEndpoints++;
 }
 if (localEndpoints  blockfor)
 throw new UnavailableException();
 {code}
 So if repair is off (so we truncate our endpoints list) AND dynamic snitch 
 has decided that nodes in another DC are to be preferred over local ones, 
 we'll throw UE even if all the replicas are healthy.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




svn commit: r1144409 - in /cassandra/branches/cassandra-0.8: ./ contrib/ interface/thrift/gen-java/org/apache/cassandra/thrift/ src/java/org/apache/cassandra/service/

2011-07-08 Thread jbellis
Author: jbellis
Date: Fri Jul  8 18:11:19 2011
New Revision: 1144409

URL: http://svn.apache.org/viewvc?rev=1144409view=rev
Log:
merge from 0.7

Modified:
cassandra/branches/cassandra-0.8/   (props changed)
cassandra/branches/cassandra-0.8/CHANGES.txt
cassandra/branches/cassandra-0.8/contrib/   (props changed)

cassandra/branches/cassandra-0.8/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
   (props changed)

cassandra/branches/cassandra-0.8/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java
   (props changed)

cassandra/branches/cassandra-0.8/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java
   (props changed)

cassandra/branches/cassandra-0.8/interface/thrift/gen-java/org/apache/cassandra/thrift/NotFoundException.java
   (props changed)

cassandra/branches/cassandra-0.8/interface/thrift/gen-java/org/apache/cassandra/thrift/SuperColumn.java
   (props changed)

cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/service/DatacenterReadCallback.java

cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/service/ReadCallback.java

Propchange: cassandra/branches/cassandra-0.8/
--
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Jul  8 18:11:19 2011
@@ -1,5 +1,5 @@
 
/cassandra/branches/cassandra-0.6:922689-1052356,1052358-1053452,1053454,1053456-1131291
-/cassandra/branches/cassandra-0.7:1026516-1143476
+/cassandra/branches/cassandra-0.7:1026516-1144407
 /cassandra/branches/cassandra-0.7.0:1053690-1055654
 /cassandra/branches/cassandra-0.8:1090934-1125013,1125041
 /cassandra/branches/cassandra-0.8.0:1125021-1130369

Modified: cassandra/branches/cassandra-0.8/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/CHANGES.txt?rev=1144409r1=1144408r2=1144409view=diff
==
--- cassandra/branches/cassandra-0.8/CHANGES.txt (original)
+++ cassandra/branches/cassandra-0.8/CHANGES.txt Fri Jul  8 18:11:19 2011
@@ -18,6 +18,11 @@
  * handle row tombstones correctly in EchoedRow (CASSANDRA-2786)
  * add MessagingService.get[Recently]DroppedMessages and
StorageService.getExceptionCount (CASSANDRA-2804)
+ * fix possibility of spurious UnavailableException for LOCAL_QUORUM
+   reads with dynamic snitch + read repair disabled (CASSANDRA-2870)
+ * add ant-optional as dependence for the debian package (CASSANDRA-2164)
+ * add option to specify limit for get_slice in the CLI (CASSANDRA-2646)
+ * decrease HH page size (CASSANDRA-2832)
 
 
 0.8.1

Propchange: cassandra/branches/cassandra-0.8/contrib/
--
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Jul  8 18:11:19 2011
@@ -1,5 +1,5 @@
 
/cassandra/branches/cassandra-0.6/contrib:922689-1052356,1052358-1053452,1053454,1053456-1068009
-/cassandra/branches/cassandra-0.7/contrib:1026516-1143476
+/cassandra/branches/cassandra-0.7/contrib:1026516-1144407
 /cassandra/branches/cassandra-0.7.0/contrib:1053690-1055654
 /cassandra/branches/cassandra-0.8/contrib:1090934-1125013,1125041
 /cassandra/branches/cassandra-0.8.0/contrib:1125021-1130369

Propchange: 
cassandra/branches/cassandra-0.8/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
--
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Jul  8 18:11:19 2011
@@ -1,5 +1,5 @@
 
/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:922689-1052356,1052358-1053452,1053454,1053456-1131291
-/cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:1026516-1143476
+/cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:1026516-1144407
 
/cassandra/branches/cassandra-0.7.0/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:1053690-1055654
 
/cassandra/branches/cassandra-0.8/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:1090934-1125013,1125041
 
/cassandra/branches/cassandra-0.8.0/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:1125021-1130369

Propchange: 
cassandra/branches/cassandra-0.8/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java
--
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Jul  8 18:11:19 2011
@@ -1,5 +1,5 @@
 
/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:922689-1052356,1052358-1053452,1053454,1053456-1131291
-/cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:1026516-1143476

svn commit: r1144411 - in /cassandra/trunk: ./ contrib/ interface/thrift/gen-java/org/apache/cassandra/thrift/ src/java/org/apache/cassandra/cli/ src/java/org/apache/cassandra/config/ src/java/org/apa

2011-07-08 Thread jbellis
Author: jbellis
Date: Fri Jul  8 18:13:45 2011
New Revision: 1144411

URL: http://svn.apache.org/viewvc?rev=1144411view=rev
Log:
merge from 0.8

Modified:
cassandra/trunk/   (props changed)
cassandra/trunk/CHANGES.txt
cassandra/trunk/contrib/   (props changed)

cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
   (props changed)

cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java
   (props changed)

cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java
   (props changed)

cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/NotFoundException.java
   (props changed)

cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/SuperColumn.java
   (props changed)
cassandra/trunk/src/java/org/apache/cassandra/cli/CliClient.java
cassandra/trunk/src/java/org/apache/cassandra/cli/CliMain.java
cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
cassandra/trunk/src/java/org/apache/cassandra/db/Table.java

cassandra/trunk/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java

cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
cassandra/trunk/src/java/org/apache/cassandra/dht/RandomPartitioner.java
cassandra/trunk/src/java/org/apache/cassandra/dht/Token.java
cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java
cassandra/trunk/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
cassandra/trunk/src/java/org/apache/cassandra/net/MessagingService.java
cassandra/trunk/src/java/org/apache/cassandra/net/MessagingServiceMBean.java

cassandra/trunk/src/java/org/apache/cassandra/service/AbstractCassandraDaemon.java

cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterReadCallback.java
cassandra/trunk/src/java/org/apache/cassandra/service/GCInspector.java
cassandra/trunk/src/java/org/apache/cassandra/service/ReadCallback.java
cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java

cassandra/trunk/src/java/org/apache/cassandra/service/StorageServiceMBean.java
cassandra/trunk/src/java/org/apache/cassandra/tools/NodeCmd.java
cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java
cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java

Propchange: cassandra/trunk/
--
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Jul  8 18:13:45 2011
@@ -1,7 +1,7 @@
 
/cassandra/branches/cassandra-0.6:922689-1052356,1052358-1053452,1053454,1053456-1131291
-/cassandra/branches/cassandra-0.7:1026516-1140567,1140928,1141129,1141213,1141217,1143437,1143444
+/cassandra/branches/cassandra-0.7:1026516-1144407
 /cassandra/branches/cassandra-0.7.0:1053690-1055654
-/cassandra/branches/cassandra-0.8:1090934-1125013,1125019-1141220,1143352,1143438,1143445
+/cassandra/branches/cassandra-0.8:1090934-1125013,1125019-1144410
 /cassandra/branches/cassandra-0.8.0:1125021-1130369
 /cassandra/branches/cassandra-0.8.1:1101014-1125018
 /cassandra/tags/cassandra-0.7.0-rc3:1051699-1053689

Modified: cassandra/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/CHANGES.txt?rev=1144411r1=1144410r2=1144411view=diff
==
--- cassandra/trunk/CHANGES.txt (original)
+++ cassandra/trunk/CHANGES.txt Fri Jul  8 18:13:45 2011
@@ -26,7 +26,23 @@
(CASSANDRA-2823)
  * Fix race in SystemTable.getCurrentLocalNodeId (CASSANDRA-2824)
  * Correctly set default for replicate_on_write (CASSANDRA-2835)
+ * improve nodetool compactionstats formatting (CASSANDRA-2844)
+ * fix index-building status display (CASSANDRA-2853)
+ * fix CLI perpetuating obsolete KsDef.replication_factor (CASSANDRA-2846)
+ * improve cli treatment of multiline comments (CASSANDRA-2852)
  * handle row tombstones correctly in EchoedRow (CASSANDRA-2786)
+ .working
+ * add MessagingService.get[Recently]DroppedMessages and
+   StorageService.getExceptionCount (CASSANDRA-2804)
+ * fix possibility of spurious UnavailableException for LOCAL_QUORUM
+   reads with dynamic snitch + read repair disabled (CASSANDRA-2870)
+ * add ant-optional as dependence for the debian package (CASSANDRA-2164)
+ * add option to specify limit for get_slice in the CLI (CASSANDRA-2646)
+ * decrease HH page size (CASSANDRA-2832)
+===
+ * add MessagingService.get[Recently]DroppedMessages and
+   StorageService.getExceptionCount (CASSANDRA-2804)
+ .merge-right.r1143437
 
 
 0.8.1
@@ -109,6 +125,7 @@
  * Always use even distribution for merkle tree with RandomPartitionner
(CASSANDRA-2841)
  * fix describeOwnership for OPP (CASSANDRA-2800)
+ * ensure that string tokens do not contain commas (CASSANDRA-2762)
 
 
 0.8.0-final

Propchange: cassandra/trunk/contrib/

svn commit: r1144412 - in /cassandra/branches/cassandra-0.8: src/java/org/apache/cassandra/cli/ src/java/org/apache/cassandra/db/ src/java/org/apache/cassandra/hadoop/ src/java/org/apache/cassandra/se

2011-07-08 Thread jbellis
Author: jbellis
Date: Fri Jul  8 18:15:49 2011
New Revision: 1144412

URL: http://svn.apache.org/viewvc?rev=1144412view=rev
Log:
r/m @author tags

Modified:

cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliUserHelp.java

cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/TruncateResponse.java

cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/Truncation.java

cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java

cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java

cassandra/branches/cassandra-0.8/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java

Modified: 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliUserHelp.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliUserHelp.java?rev=1144412r1=1144411r2=1144412view=diff
==
--- 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliUserHelp.java
 (original)
+++ 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliUserHelp.java
 Fri Jul  8 18:15:49 2011
@@ -19,9 +19,6 @@ package org.apache.cassandra.cli;
 
 import java.util.List;
 
-/**
- * @author Pavel A. Yaskevich
- */
 public class CliUserHelp
 {
 public String banner;

Modified: 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/TruncateResponse.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/TruncateResponse.java?rev=1144412r1=1144411r2=1144412view=diff
==
--- 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/TruncateResponse.java
 (original)
+++ 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/TruncateResponse.java
 Fri Jul  8 18:15:49 2011
@@ -31,8 +31,6 @@ import org.apache.cassandra.utils.FBUtil
 /**
  * This message is sent back the truncate operation and basically specifies if
  * the truncate succeeded.
- *
- * @author ran...@gmail.com
  */
 public class TruncateResponse
 {

Modified: 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/Truncation.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/Truncation.java?rev=1144412r1=1144411r2=1144412view=diff
==
--- 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/Truncation.java
 (original)
+++ 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/Truncation.java
 Fri Jul  8 18:15:49 2011
@@ -31,9 +31,6 @@ import org.apache.cassandra.utils.FBUtil
 
 /**
  * A truncate operation descriptor
- *
- * @author ran...@gmail.com
- *
  */
 public class Truncation implements MessageProducer
 {

Modified: 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java?rev=1144412r1=1144411r2=1144412view=diff
==
--- 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
 (original)
+++ 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
 Fri Jul  8 18:15:49 2011
@@ -53,7 +53,6 @@ import org.apache.thrift.transport.TSock
  * directly to a responsible endpoint.
  * /p
  * 
- * @author Karthick Sankarachary
  * @see ColumnFamilyOutputFormat
  * @see OutputFormat
  * 

Modified: 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java?rev=1144412r1=1144411r2=1144412view=diff
==
--- 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
 (original)
+++ 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
 Fri Jul  8 18:15:49 2011
@@ -45,8 +45,6 @@ import org.apache.thrift.transport.TTran
 cassandra.start();
 
  * /pre
- * @author Ran Tavory (ran...@gmail.com)
- *
  */
 public class EmbeddedCassandraService
 {

Modified: 
cassandra/branches/cassandra-0.8/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java?rev=1144412r1=1144411r2=1144412view=diff

[jira] [Commented] (CASSANDRA-2810) RuntimeException in Pig when using dump command on column name

2011-07-08 Thread Brandon Williams (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2810?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062128#comment-13062128
 ] 

Brandon Williams commented on CASSANDRA-2810:
-

So is the conclusion that this patch by itself works fine, but there is a 
problem with CASSANDRA-2777?

 RuntimeException in Pig when using dump command on column name
 

 Key: CASSANDRA-2810
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2810
 Project: Cassandra
  Issue Type: Bug
Affects Versions: 0.8.1
 Environment: Ubuntu 10.10, 32 bits
 java version 1.6.0_24
 Brisk beta-2 installed from Debian packages
Reporter: Silvère Lestang
Assignee: Brandon Williams
 Attachments: 2810.txt


 This bug was previously report on [Brisk bug 
 tracker|https://datastax.jira.com/browse/BRISK-232].
 In cassandra-cli:
 {code}
 [default@unknown] create keyspace Test
 with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy'
 and strategy_options = [{replication_factor:1}];
 [default@unknown] use Test;
 Authenticated to keyspace: Test
 [default@Test] create column family test;
 [default@Test] set test[ascii('row1')][long(1)]=integer(35);
 set test[ascii('row1')][long(2)]=integer(36);
 set test[ascii('row1')][long(3)]=integer(38);
 set test[ascii('row2')][long(1)]=integer(45);
 set test[ascii('row2')][long(2)]=integer(42);
 set test[ascii('row2')][long(3)]=integer(33);
 [default@Test] list test;
 Using default limit of 100
 ---
 RowKey: 726f7731
 = (column=0001, value=35, timestamp=1308744931122000)
 = (column=0002, value=36, timestamp=1308744931124000)
 = (column=0003, value=38, timestamp=1308744931125000)
 ---
 RowKey: 726f7732
 = (column=0001, value=45, timestamp=1308744931127000)
 = (column=0002, value=42, timestamp=1308744931128000)
 = (column=0003, value=33, timestamp=1308744932722000)
 2 Rows Returned.
 [default@Test] describe keyspace;
 Keyspace: Test:
   Replication Strategy: org.apache.cassandra.locator.SimpleStrategy
   Durable Writes: true
 Options: [replication_factor:1]
   Column Families:
 ColumnFamily: test
   Key Validation Class: org.apache.cassandra.db.marshal.BytesType
   Default column value validator: 
 org.apache.cassandra.db.marshal.BytesType
   Columns sorted by: org.apache.cassandra.db.marshal.BytesType
   Row cache size / save period in seconds: 0.0/0
   Key cache size / save period in seconds: 20.0/14400
   Memtable thresholds: 0.571875/122/1440 (millions of ops/MB/minutes)
   GC grace seconds: 864000
   Compaction min/max thresholds: 4/32
   Read repair chance: 1.0
   Replicate on write: false
   Built indexes: []
 {code}
 In Pig command line:
 {code}
 grunt test = LOAD 'cassandra://Test/test' USING CassandraStorage() AS 
 (rowkey:chararray, columns: bag {T: (name:long, value:int)});
 grunt value_test = foreach test generate rowkey, columns.name, columns.value;
 grunt dump value_test;
 {code}
 In /var/log/cassandra/system.log, I have severals time this exception:
 {code}
 INFO [IPC Server handler 3 on 8012] 2011-06-22 15:03:28,533 
 TaskInProgress.java (line 551) Error from 
 attempt_201106210955_0051_m_00_3: java.lang.RuntimeException: Unexpected 
 data type -1 found in stream.
   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:478)
   at org.apache.pig.data.BinInterSedes.writeTuple(BinInterSedes.java:541)
   at org.apache.pig.data.BinInterSedes.writeBag(BinInterSedes.java:522)
   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:361)
   at org.apache.pig.data.BinInterSedes.writeTuple(BinInterSedes.java:541)
   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:357)
   at 
 org.apache.pig.impl.io.InterRecordWriter.write(InterRecordWriter.java:73)
   at org.apache.pig.impl.io.InterStorage.putNext(InterStorage.java:87)
   at 
 org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigOutputFormat$PigRecordWriter.write(PigOutputFormat.java:138)
   at 
 org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigOutputFormat$PigRecordWriter.write(PigOutputFormat.java:97)
   at 
 org.apache.hadoop.mapred.MapTask$NewDirectOutputCollector.write(MapTask.java:638)
   at 
 org.apache.hadoop.mapreduce.TaskInputOutputContext.write(TaskInputOutputContext.java:80)
   at 
 org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapOnly$Map.collect(PigMapOnly.java:48)
   at 
 org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.runPipeline(PigMapBase.java:239)
   at 
 

[jira] [Commented] (CASSANDRA-2791) Redhat spec file needs some enhancements for 0.8 and beyond

2011-07-08 Thread Niels Basjes (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2791?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062177#comment-13062177
 ] 

Niels Basjes commented on CASSANDRA-2791:
-

Could you please check if the patch I created for CASSANDRA-2861 solves all 
issues for you?

 Redhat spec file needs some enhancements for 0.8 and beyond
 ---

 Key: CASSANDRA-2791
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2791
 Project: Cassandra
  Issue Type: Improvement
  Components: Packaging
Affects Versions: 0.8.0
Reporter: Nate McCall
 Fix For: 0.8.2, 1.0


 Version and Release need to be brought up to date. Also need to account for 
 multiple 'apache-cassandra' jars. 

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2863) NPE when writing SSTable generated via repair

2011-07-08 Thread JIRA

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062191#comment-13062191
 ] 

Héctor Izquierdo commented on CASSANDRA-2863:
-

I don't now if it's the same one, buy I got another during repair on another 
node:

ERROR [Thread-1710] 2011-07-08 21:21:00,514 AbstractCassandraDaemon.java (line 
113) Fatal exception in thread Thread[Thread-1710,5,main]
java.lang.RuntimeException: java.util.concurrent.ExecutionException: 
java.lang.NullPointerException
at 
org.apache.cassandra.streaming.StreamInSession.closeIfFinished(StreamInSession.java:154)
at 
org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:63)
at 
org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:162)
at 
org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:95)
Caused by: java.util.concurrent.ExecutionException: 
java.lang.NullPointerException
at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222)
at java.util.concurrent.FutureTask.get(FutureTask.java:83)
at 
org.apache.cassandra.streaming.StreamInSession.closeIfFinished(StreamInSession.java:138)
... 3 more
Caused by: java.lang.NullPointerException
at 
org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.close(SSTableWriter.java:382)
at 
org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.index(SSTableWriter.java:370)
at 
org.apache.cassandra.io.sstable.SSTableWriter$Builder.build(SSTableWriter.java:315)
at 
org.apache.cassandra.db.compaction.CompactionManager$9.call(CompactionManager.java:1103)
at 
org.apache.cassandra.db.compaction.CompactionManager$9.call(CompactionManager.java:1094)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
at java.util.concurrent.FutureTask.run(FutureTask.java:138)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
at java.lang.Thread.run(Thread.java:662)


 NPE when writing SSTable generated via repair
 -

 Key: CASSANDRA-2863
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2863
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.8.1
Reporter: Héctor Izquierdo
Assignee: Sylvain Lebresne
 Fix For: 0.8.2


 A NPE is generated during repair when closing an sstable generated via 
 SSTable build. It doesn't happen always. The node had been scrubbed and 
 compacted before calling repair.
  INFO [CompactionExecutor:2] 2011-07-06 11:11:32,640 SSTableReader.java (line 
 158) Opening /d2/cassandra/data/sbs/walf-g-730
 ERROR [CompactionExecutor:2] 2011-07-06 11:11:34,327 
 AbstractCassandraDaemon.java (line 113) Fatal exception in thread 
 Thread[CompactionExecutor:2,1,main] 
 java.lang.NullPointerException
   at 
 org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.close(SSTableWriter.java:382)
   at 
 org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.index(SSTableWriter.java:370)
   at 
 org.apache.cassandra.io.sstable.SSTableWriter$Builder.build(SSTableWriter.java:315)
   at 
 org.apache.cassandra.db.compaction.CompactionManager$9.call(CompactionManager.java:1103)
   at 
 org.apache.cassandra.db.compaction.CompactionManager$9.call(CompactionManager.java:1094)
   at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
   at java.util.concurrent.FutureTask.run(FutureTask.java:138)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
   at java.lang.Thread.run(Thread.java:662)

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (CASSANDRA-47) SSTable compression

2011-07-08 Thread Pavel Yaskevich (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Pavel Yaskevich updated CASSANDRA-47:
-

Attachment: snappy-java-1.0.3-rc4.jar
CASSANDRA-47.patch

Patch introduces CompressedDataFile with Input/Output classes. Snappy is used 
for compression/decompression because it showed better speeds in tests 
comparing to ning. Files are split into 4 bytes + 64kb chunks where 4 bytes 
hold information about compressed chunk size. Both Input and Output classes 
extend RandomAccessFile so random I/O works as expected.

All SSTable files are opened using CompressedDataFile.Input. On startup when 
SSTableReader.open gets called it first checks if data file is already 
compressed and compresses if it was not already compressed so users won't have 
a problem after they update.

At the header of the file it reserves 8 bytes for a real data size so other 
components of the system that use SSTables and SSTables itself have no idea 
that data file is compressed.

Streaming of data file sends decompressed chunks for convenience of maintaing 
transfer and receiving party compresses all data before write to the backing 
file (see CompressedDataFile.transfer(...) and CompressedFileReceiver class).

Tests are showing dramatic performance increase when reading 1 million rows 
created with 1024 bytes random values. Current code takes  1000 secs to read 
but with current path only 175 secs. Using 64kb buffer 1.7GB file could be 
compressed into 110MB (data added using ./bin/stress -n 100 -S 1024 -r, 
where -r option generates random values).

Writes perform a bit better like 5-10%. 

 SSTable compression
 ---

 Key: CASSANDRA-47
 URL: https://issues.apache.org/jira/browse/CASSANDRA-47
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Jonathan Ellis
Assignee: Pavel Yaskevich
  Labels: compression
 Fix For: 1.0

 Attachments: CASSANDRA-47.patch, snappy-java-1.0.3-rc4.jar


 We should be able to do SSTable compression which would trade CPU for I/O 
 (almost always a good trade).

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Issue Comment Edited] (CASSANDRA-47) SSTable compression

2011-07-08 Thread Pavel Yaskevich (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062204#comment-13062204
 ] 

Pavel Yaskevich edited comment on CASSANDRA-47 at 7/8/11 9:46 PM:
--

Patch introduces CompressedDataFile with Input/Output classes. Snappy is used 
for compression/decompression because it showed better speeds in tests 
comparing to ning. Files are split into 4 bytes + 64kb chunks where 4 bytes 
hold information about compressed chunk size, not that current SSTable file 
format is preserved and no modifications were made to index, statistics or 
filter components. Both Input and Output classes extend RandomAccessFile so 
random I/O works as expected.

All SSTable files are opened using CompressedDataFile.Input. On startup when 
SSTableReader.open gets called it first checks if data file is already 
compressed and compresses if it was not already compressed so users won't have 
a problem after they update.

At the header of the file it reserves 8 bytes for a real data size so other 
components of the system that use SSTables and SSTables itself have no idea 
that data file is compressed.

Streaming of data file sends decompressed chunks for convenience of maintaing 
transfer and receiving party compresses all data before write to the backing 
file (see CompressedDataFile.transfer(...) and CompressedFileReceiver class).

Tests are showing dramatic performance increase when reading 1 million rows 
created with 1024 bytes random values. Current code takes  1000 secs to read 
but with current path only 175 secs. Using 64kb buffer 1.7GB file could be 
compressed into 110MB (data added using ./bin/stress -n 100 -S 1024 -r, 
where -r option generates random values).

Writes perform a bit better like 5-10%. 

  was (Author: xedin):
Patch introduces CompressedDataFile with Input/Output classes. Snappy is 
used for compression/decompression because it showed better speeds in tests 
comparing to ning. Files are split into 4 bytes + 64kb chunks where 4 bytes 
hold information about compressed chunk size. Both Input and Output classes 
extend RandomAccessFile so random I/O works as expected.

All SSTable files are opened using CompressedDataFile.Input. On startup when 
SSTableReader.open gets called it first checks if data file is already 
compressed and compresses if it was not already compressed so users won't have 
a problem after they update.

At the header of the file it reserves 8 bytes for a real data size so other 
components of the system that use SSTables and SSTables itself have no idea 
that data file is compressed.

Streaming of data file sends decompressed chunks for convenience of maintaing 
transfer and receiving party compresses all data before write to the backing 
file (see CompressedDataFile.transfer(...) and CompressedFileReceiver class).

Tests are showing dramatic performance increase when reading 1 million rows 
created with 1024 bytes random values. Current code takes  1000 secs to read 
but with current path only 175 secs. Using 64kb buffer 1.7GB file could be 
compressed into 110MB (data added using ./bin/stress -n 100 -S 1024 -r, 
where -r option generates random values).

Writes perform a bit better like 5-10%. 
  
 SSTable compression
 ---

 Key: CASSANDRA-47
 URL: https://issues.apache.org/jira/browse/CASSANDRA-47
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Jonathan Ellis
Assignee: Pavel Yaskevich
  Labels: compression
 Fix For: 1.0

 Attachments: CASSANDRA-47.patch, snappy-java-1.0.3-rc4.jar


 We should be able to do SSTable compression which would trade CPU for I/O 
 (almost always a good trade).

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-47) SSTable compression

2011-07-08 Thread Stu Hood (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062221#comment-13062221
 ] 

Stu Hood commented on CASSANDRA-47:
---

bq. where -r option generates random values
The -r flag generates random keys: unless you modified stress.java, the values 
will be the same for every row.

 SSTable compression
 ---

 Key: CASSANDRA-47
 URL: https://issues.apache.org/jira/browse/CASSANDRA-47
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Jonathan Ellis
Assignee: Pavel Yaskevich
  Labels: compression
 Fix For: 1.0

 Attachments: CASSANDRA-47.patch, snappy-java-1.0.3-rc4.jar


 We should be able to do SSTable compression which would trade CPU for I/O 
 (almost always a good trade).

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-47) SSTable compression

2011-07-08 Thread Pavel Yaskevich (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062228#comment-13062228
 ] 

Pavel Yaskevich commented on CASSANDRA-47:
--

bq. The -r flag generates random keys: unless you modified stress.java, the 
values will be the same for every row.

CASSANDRA-2266 . Also used various cardinality 50-250 in the tests

 SSTable compression
 ---

 Key: CASSANDRA-47
 URL: https://issues.apache.org/jira/browse/CASSANDRA-47
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Jonathan Ellis
Assignee: Pavel Yaskevich
  Labels: compression
 Fix For: 1.0

 Attachments: CASSANDRA-47.patch, snappy-java-1.0.3-rc4.jar


 We should be able to do SSTable compression which would trade CPU for I/O 
 (almost always a good trade).

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Issue Comment Edited] (CASSANDRA-47) SSTable compression

2011-07-08 Thread Pavel Yaskevich (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062228#comment-13062228
 ] 

Pavel Yaskevich edited comment on CASSANDRA-47 at 7/8/11 10:21 PM:
---

bq. The -r flag generates random keys: unless you modified stress.java, the 
values will be the same for every row.

oh, sorry! I meant -V not -r also used various cardinality 50-250 in the tests

  was (Author: xedin):
bq. The -r flag generates random keys: unless you modified stress.java, the 
values will be the same for every row.

CASSANDRA-2266 . Also used various cardinality 50-250 in the tests
  
 SSTable compression
 ---

 Key: CASSANDRA-47
 URL: https://issues.apache.org/jira/browse/CASSANDRA-47
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Jonathan Ellis
Assignee: Pavel Yaskevich
  Labels: compression
 Fix For: 1.0

 Attachments: CASSANDRA-47.patch, snappy-java-1.0.3-rc4.jar


 We should be able to do SSTable compression which would trade CPU for I/O 
 (almost always a good trade).

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Issue Comment Edited] (CASSANDRA-47) SSTable compression

2011-07-08 Thread Pavel Yaskevich (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062204#comment-13062204
 ] 

Pavel Yaskevich edited comment on CASSANDRA-47 at 7/8/11 10:24 PM:
---

Patch introduces CompressedDataFile with Input/Output classes. Snappy is used 
for compression/decompression because it showed better speeds in tests 
comparing to ning. Files are split into 4 bytes + 64kb chunks where 4 bytes 
hold information about compressed chunk size, not that current SSTable file 
format is preserved and no modifications were made to index, statistics or 
filter components. Both Input and Output classes extend RandomAccessFile so 
random I/O works as expected.

All SSTable files are opened using CompressedDataFile.Input. On startup when 
SSTableReader.open gets called it first checks if data file is already 
compressed and compresses if it was not already compressed so users won't have 
a problem after they update.

At the header of the file it reserves 8 bytes for a real data size so other 
components of the system that use SSTables and SSTables itself have no idea 
that data file is compressed.

Streaming of data file sends decompressed chunks for convenience of maintaing 
transfer and receiving party compresses all data before write to the backing 
file (see CompressedDataFile.transfer(...) and CompressedFileReceiver class).

Tests are showing dramatic performance increase when reading 1 million rows 
created with 1024 bytes random values. Current code takes  1000 secs to read 
but with current path only 175 secs. Using 64kb buffer 1.7GB file could be 
compressed into 110MB (data added using ./bin/stress -n 100 -S 1024 -V, 
where -V option generates average size values and different cardinality from 50 
(default) to 250).

Writes perform a bit better like 5-10%. 

  was (Author: xedin):
Patch introduces CompressedDataFile with Input/Output classes. Snappy is 
used for compression/decompression because it showed better speeds in tests 
comparing to ning. Files are split into 4 bytes + 64kb chunks where 4 bytes 
hold information about compressed chunk size, not that current SSTable file 
format is preserved and no modifications were made to index, statistics or 
filter components. Both Input and Output classes extend RandomAccessFile so 
random I/O works as expected.

All SSTable files are opened using CompressedDataFile.Input. On startup when 
SSTableReader.open gets called it first checks if data file is already 
compressed and compresses if it was not already compressed so users won't have 
a problem after they update.

At the header of the file it reserves 8 bytes for a real data size so other 
components of the system that use SSTables and SSTables itself have no idea 
that data file is compressed.

Streaming of data file sends decompressed chunks for convenience of maintaing 
transfer and receiving party compresses all data before write to the backing 
file (see CompressedDataFile.transfer(...) and CompressedFileReceiver class).

Tests are showing dramatic performance increase when reading 1 million rows 
created with 1024 bytes random values. Current code takes  1000 secs to read 
but with current path only 175 secs. Using 64kb buffer 1.7GB file could be 
compressed into 110MB (data added using ./bin/stress -n 100 -S 1024 -V, 
where -V option generates random values).

Writes perform a bit better like 5-10%. 
  
 SSTable compression
 ---

 Key: CASSANDRA-47
 URL: https://issues.apache.org/jira/browse/CASSANDRA-47
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Jonathan Ellis
Assignee: Pavel Yaskevich
  Labels: compression
 Fix For: 1.0

 Attachments: CASSANDRA-47.patch, snappy-java-1.0.3-rc4.jar


 We should be able to do SSTable compression which would trade CPU for I/O 
 (almost always a good trade).

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Issue Comment Edited] (CASSANDRA-47) SSTable compression

2011-07-08 Thread Pavel Yaskevich (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062204#comment-13062204
 ] 

Pavel Yaskevich edited comment on CASSANDRA-47 at 7/8/11 10:22 PM:
---

Patch introduces CompressedDataFile with Input/Output classes. Snappy is used 
for compression/decompression because it showed better speeds in tests 
comparing to ning. Files are split into 4 bytes + 64kb chunks where 4 bytes 
hold information about compressed chunk size, not that current SSTable file 
format is preserved and no modifications were made to index, statistics or 
filter components. Both Input and Output classes extend RandomAccessFile so 
random I/O works as expected.

All SSTable files are opened using CompressedDataFile.Input. On startup when 
SSTableReader.open gets called it first checks if data file is already 
compressed and compresses if it was not already compressed so users won't have 
a problem after they update.

At the header of the file it reserves 8 bytes for a real data size so other 
components of the system that use SSTables and SSTables itself have no idea 
that data file is compressed.

Streaming of data file sends decompressed chunks for convenience of maintaing 
transfer and receiving party compresses all data before write to the backing 
file (see CompressedDataFile.transfer(...) and CompressedFileReceiver class).

Tests are showing dramatic performance increase when reading 1 million rows 
created with 1024 bytes random values. Current code takes  1000 secs to read 
but with current path only 175 secs. Using 64kb buffer 1.7GB file could be 
compressed into 110MB (data added using ./bin/stress -n 100 -S 1024 -V, 
where -V option generates random values).

Writes perform a bit better like 5-10%. 

  was (Author: xedin):
Patch introduces CompressedDataFile with Input/Output classes. Snappy is 
used for compression/decompression because it showed better speeds in tests 
comparing to ning. Files are split into 4 bytes + 64kb chunks where 4 bytes 
hold information about compressed chunk size, not that current SSTable file 
format is preserved and no modifications were made to index, statistics or 
filter components. Both Input and Output classes extend RandomAccessFile so 
random I/O works as expected.

All SSTable files are opened using CompressedDataFile.Input. On startup when 
SSTableReader.open gets called it first checks if data file is already 
compressed and compresses if it was not already compressed so users won't have 
a problem after they update.

At the header of the file it reserves 8 bytes for a real data size so other 
components of the system that use SSTables and SSTables itself have no idea 
that data file is compressed.

Streaming of data file sends decompressed chunks for convenience of maintaing 
transfer and receiving party compresses all data before write to the backing 
file (see CompressedDataFile.transfer(...) and CompressedFileReceiver class).

Tests are showing dramatic performance increase when reading 1 million rows 
created with 1024 bytes random values. Current code takes  1000 secs to read 
but with current path only 175 secs. Using 64kb buffer 1.7GB file could be 
compressed into 110MB (data added using ./bin/stress -n 100 -S 1024 -r, 
where -r option generates random values).

Writes perform a bit better like 5-10%. 
  
 SSTable compression
 ---

 Key: CASSANDRA-47
 URL: https://issues.apache.org/jira/browse/CASSANDRA-47
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Jonathan Ellis
Assignee: Pavel Yaskevich
  Labels: compression
 Fix For: 1.0

 Attachments: CASSANDRA-47.patch, snappy-java-1.0.3-rc4.jar


 We should be able to do SSTable compression which would trade CPU for I/O 
 (almost always a good trade).

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-47) SSTable compression

2011-07-08 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062237#comment-13062237
 ] 

Jonathan Ellis commented on CASSANDRA-47:
-

How does the [row] index interact with the block implementation?

 SSTable compression
 ---

 Key: CASSANDRA-47
 URL: https://issues.apache.org/jira/browse/CASSANDRA-47
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Jonathan Ellis
Assignee: Pavel Yaskevich
  Labels: compression
 Fix For: 1.0

 Attachments: CASSANDRA-47.patch, snappy-java-1.0.3-rc4.jar


 We should be able to do SSTable compression which would trade CPU for I/O 
 (almost always a good trade).

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-47) SSTable compression

2011-07-08 Thread Pavel Yaskevich (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062240#comment-13062240
 ] 

Pavel Yaskevich commented on CASSANDRA-47:
--

It just refers to uncompressed locations, I didn't see a need to change to 
change that.

 SSTable compression
 ---

 Key: CASSANDRA-47
 URL: https://issues.apache.org/jira/browse/CASSANDRA-47
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Jonathan Ellis
Assignee: Pavel Yaskevich
  Labels: compression
 Fix For: 1.0

 Attachments: CASSANDRA-47.patch, snappy-java-1.0.3-rc4.jar


 We should be able to do SSTable compression which would trade CPU for I/O 
 (almost always a good trade).

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-47) SSTable compression

2011-07-08 Thread Benjamin Coverston (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062244#comment-13062244
 ] 

Benjamin Coverston commented on CASSANDRA-47:
-

I was worried a bit about this ticket while working on 1608 because it had the 
potential to be a hairy merge. Looking through the patch however the 
implementation looks clean, unobtrusive and well encapsulated. This makes me 
really excited. Contrary to my initial fears I'm anticipating that this will 
actually help with 1608 and probably even help increase IO throughput in the 
data directory in a dramatic way.

+1 for this approach.

 SSTable compression
 ---

 Key: CASSANDRA-47
 URL: https://issues.apache.org/jira/browse/CASSANDRA-47
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Jonathan Ellis
Assignee: Pavel Yaskevich
  Labels: compression
 Fix For: 1.0

 Attachments: CASSANDRA-47.patch, snappy-java-1.0.3-rc4.jar


 We should be able to do SSTable compression which would trade CPU for I/O 
 (almost always a good trade).

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2870) dynamic snitch + read repair off can cause LOCAL_QUORUM reads to return spurious UnavailableException

2011-07-08 Thread Jeremy Hanna (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2870?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062256#comment-13062256
 ] 

Jeremy Hanna commented on CASSANDRA-2870:
-

This also appears to affect 0.7.6 and when read repair is not off.  I didn't 
set read repair on my CFs (defaults to 100%) and tried a simple rowcount pig 
script using read consistency LOCAL_QUORUM and it fails with UE.  I would think 
if that's the case, the priority should be higher and it should go in 0.7.7.  
Any thoughts?

 dynamic snitch + read repair off can cause LOCAL_QUORUM reads to return 
 spurious UnavailableException
 -

 Key: CASSANDRA-2870
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2870
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.0
Reporter: Jonathan Ellis
Assignee: Jonathan Ellis
Priority: Minor
 Fix For: 0.7.8, 0.8.2

 Attachments: 2870.txt


 When Read Repair is off, we want to avoid doing requests to more nodes than 
 necessary to satisfy the ConsistencyLevel.  ReadCallback does this here:
 {code}
 this.endpoints = repair || resolver instanceof RowRepairResolver
? endpoints
: endpoints.subList(0, Math.min(endpoints.size(), 
 blockfor)); // min so as to not throw exception until assureSufficient is 
 called
 {code}
 You can see that it is assuming that the endpoints list is sorted in order 
 of preferred-ness for the read.
 Then the LOCAL_QUORUM code in DatacenterReadCallback checks to see if we have 
 enough nodes to do the read:
 {code}
 int localEndpoints = 0;
 for (InetAddress endpoint : endpoints)
 {
 if (localdc.equals(snitch.getDatacenter(endpoint)))
 localEndpoints++;
 }
 if (localEndpoints  blockfor)
 throw new UnavailableException();
 {code}
 So if repair is off (so we truncate our endpoints list) AND dynamic snitch 
 has decided that nodes in another DC are to be preferred over local ones, 
 we'll throw UE even if all the replicas are healthy.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Issue Comment Edited] (CASSANDRA-47) SSTable compression

2011-07-08 Thread Pavel Yaskevich (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062240#comment-13062240
 ] 

Pavel Yaskevich edited comment on CASSANDRA-47 at 7/9/11 12:14 AM:
---

It just refers to uncompressed locations, I didn't see a need to change that.

  was (Author: xedin):
It just refers to uncompressed locations, I didn't see a need to change to 
change that.
  
 SSTable compression
 ---

 Key: CASSANDRA-47
 URL: https://issues.apache.org/jira/browse/CASSANDRA-47
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Jonathan Ellis
Assignee: Pavel Yaskevich
  Labels: compression
 Fix For: 1.0

 Attachments: CASSANDRA-47.patch, snappy-java-1.0.3-rc4.jar


 We should be able to do SSTable compression which would trade CPU for I/O 
 (almost always a good trade).

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Created] (CASSANDRA-2874) commitlogs are not draining

2011-07-08 Thread Scott Dworkis (JIRA)
commitlogs are not draining
---

 Key: CASSANDRA-2874
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2874
 Project: Cassandra
  Issue Type: Bug
  Components: Tools
Affects Versions: 0.8.0
 Environment: nodetool
Reporter: Scott Dworkis
Priority: Minor


i have commitlogs on every node in the ring that will not drain after invoking 
a drain with nodetool.  i'll attach one of the log headers at the request of 
Aaron Morton.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (CASSANDRA-2874) commitlogs are not draining

2011-07-08 Thread Scott Dworkis (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-2874?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Scott Dworkis updated CASSANDRA-2874:
-

Attachment: CommitLog-1308192929288.log.header

this is a 44 byte header from a 466 byte commitlog that won't drain or change 
size.

 commitlogs are not draining
 ---

 Key: CASSANDRA-2874
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2874
 Project: Cassandra
  Issue Type: Bug
  Components: Tools
Affects Versions: 0.8.0
 Environment: nodetool
Reporter: Scott Dworkis
Priority: Minor
 Attachments: CommitLog-1308192929288.log.header


 i have commitlogs on every node in the ring that will not drain after 
 invoking a drain with nodetool.  i'll attach one of the log headers at the 
 request of Aaron Morton.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2874) commitlogs are not draining

2011-07-08 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2874?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062268#comment-13062268
 ] 

Jonathan Ellis commented on CASSANDRA-2874:
---

what does it log at debug level?

i'm guessing something like Not safe to delete commit log... ?

 commitlogs are not draining
 ---

 Key: CASSANDRA-2874
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2874
 Project: Cassandra
  Issue Type: Bug
  Components: Tools
Affects Versions: 0.8.0
 Environment: nodetool
Reporter: Scott Dworkis
Priority: Minor
 Attachments: CommitLog-1308192929288.log.header


 i have commitlogs on every node in the ring that will not drain after 
 invoking a drain with nodetool.  i'll attach one of the log headers at the 
 request of Aaron Morton.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2491) A new config parameter, broadcast_address

2011-07-08 Thread Khee Chin (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2491?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062294#comment-13062294
 ] 

Khee Chin commented on CASSANDRA-2491:
--

Thanks Brandon and Vijay.

 A new config parameter, broadcast_address
 -

 Key: CASSANDRA-2491
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2491
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
 Environment: x86_64 GNU/Linux
Reporter: Khee Chin
Assignee: Vijay
Priority: Trivial
  Labels: patch
 Fix For: 1.0

 Attachments: 0001-2491-Git_Patch_v7.patch, 
 2491_broadcast_address.patch, 2491_broadcast_address_v6.patch

   Original Estimate: 336h
  Remaining Estimate: 336h

 A new config parameter, broadcast_address
 In a cluster setup where one or more nodes is behind a firewall and has a 
 private ip address, listen_address does not allow the hosts behind the 
 firewalls to be discovered by other nodes.
 Attached is a patch that introduces a new config parameter broadcast_address 
 which allows Cassandra nodes to explicitly specify their external ip address. 
 In addition, this allows listen_address to be set to 0.0.0.0 on the already 
 firewalled node.
 broadcast_address fallsback to listen_address when it is not stated.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




svn commit: r1144586 - in /cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/compaction: CompactionController.java CompactionIterator.java

2011-07-08 Thread jbellis
Author: jbellis
Date: Sat Jul  9 03:34:02 2011
New Revision: 1144586

URL: http://svn.apache.org/viewvc?rev=1144586view=rev
Log:
make throttle resolution finer grained, especially for large rows
patch by jbellis

Modified:

cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/compaction/CompactionController.java

cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java

Modified: 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/compaction/CompactionController.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/compaction/CompactionController.java?rev=1144586r1=1144585r2=1144586view=diff
==
--- 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/compaction/CompactionController.java
 (original)
+++ 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/compaction/CompactionController.java
 Sat Jul  9 03:34:02 2011
@@ -46,6 +46,7 @@ public class CompactionController
 
 public final boolean isMajor;
 public final int gcBefore;
+private int throttleResolution;
 
 public CompactionController(ColumnFamilyStore cfs, 
CollectionSSTableReader sstables, int gcBefore, boolean forceDeserialize)
 {
@@ -55,15 +56,22 @@ public class CompactionController
 this.gcBefore = gcBefore;
 this.forceDeserialize = forceDeserialize;
 isMajor = cfs.isCompleteSSTables(this.sstables);
+// how many rows we expect to compact in 100ms
+throttleResolution = (int) 
(DatabaseDescriptor.getCompactionThroughputMbPerSec() * 1024 * 1024 / (10 * 
cfs.getMeanRowSize()));
+if (throttleResolution = 0)
+throttleResolution = 1;
+}
+
+public int getThrottleResolution()
+{
+return throttleResolution;
 }
 
-/** @return the keyspace name */
 public String getKeyspace()
 {
 return cfs.table.name;
 }
 
-/** @return the column family name */
 public String getColumnFamily()
 {
 return cfs.columnFamily;

Modified: 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java?rev=1144586r1=1144585r2=1144586view=diff
==
--- 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
 (original)
+++ 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
 Sat Jul  9 03:34:02 2011
@@ -137,13 +137,11 @@ implements Closeable, CompactionInfo.Hol
 finally
 {
 rows.clear();
-if ((row++ % 1000) == 0)
+if ((row++ % controller.getThrottleResolution()) == 0)
 {
 bytesRead = 0;
 for (SSTableScanner scanner : getScanners())
-{
 bytesRead += scanner.getFilePointer();
-}
 throttle();
 }
 }
@@ -164,9 +162,7 @@ implements Closeable, CompactionInfo.Hol
 int newTarget = totalBytesPerMS /
 Math.max(1, CompactionManager.instance.getActiveCompactions());
 if (newTarget != targetBytesPerMS)
-logger.info(String.format(%s now compacting at %d bytes/ms.,
-  this,
-  newTarget));
+logger.debug({} now compacting at {} bytes/ms., this, newTarget);
 targetBytesPerMS = newTarget;
 
 // the excess bytes that were compacted in this period
@@ -179,7 +175,14 @@ implements Closeable, CompactionInfo.Hol
 if (logger.isTraceEnabled())
 logger.trace(String.format(Compacted %d bytes in %d ms: 
throttling for %d ms,
bytesSinceLast, msSinceLast, 
timeToDelay));
-try { Thread.sleep(timeToDelay); } catch (InterruptedException e) 
{ throw new AssertionError(e); }
+try
+{
+Thread.sleep(timeToDelay);
+}
+catch (InterruptedException e)
+{
+throw new AssertionError(e);
+}
 }
 bytesAtLastDelay = bytesRead;
 timeAtLastDelay = System.currentTimeMillis();




[jira] [Commented] (CASSANDRA-2870) dynamic snitch + read repair off can cause LOCAL_QUORUM reads to return spurious UnavailableException

2011-07-08 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2870?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062308#comment-13062308
 ] 

Jonathan Ellis commented on CASSANDRA-2870:
---

This has been present since LOCAL_QUORUM was introduced, so it's not a new 
regression.  And a reasonable workaround exists (disable dynamic snitch).  So 
no, I don't think we should hold up 0.7.7 for this.

 dynamic snitch + read repair off can cause LOCAL_QUORUM reads to return 
 spurious UnavailableException
 -

 Key: CASSANDRA-2870
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2870
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.0
Reporter: Jonathan Ellis
Assignee: Jonathan Ellis
Priority: Minor
 Fix For: 0.7.8, 0.8.2

 Attachments: 2870.txt


 When Read Repair is off, we want to avoid doing requests to more nodes than 
 necessary to satisfy the ConsistencyLevel.  ReadCallback does this here:
 {code}
 this.endpoints = repair || resolver instanceof RowRepairResolver
? endpoints
: endpoints.subList(0, Math.min(endpoints.size(), 
 blockfor)); // min so as to not throw exception until assureSufficient is 
 called
 {code}
 You can see that it is assuming that the endpoints list is sorted in order 
 of preferred-ness for the read.
 Then the LOCAL_QUORUM code in DatacenterReadCallback checks to see if we have 
 enough nodes to do the read:
 {code}
 int localEndpoints = 0;
 for (InetAddress endpoint : endpoints)
 {
 if (localdc.equals(snitch.getDatacenter(endpoint)))
 localEndpoints++;
 }
 if (localEndpoints  blockfor)
 throw new UnavailableException();
 {code}
 So if repair is off (so we truncate our endpoints list) AND dynamic snitch 
 has decided that nodes in another DC are to be preferred over local ones, 
 we'll throw UE even if all the replicas are healthy.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2874) commitlogs are not draining

2011-07-08 Thread Scott Dworkis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2874?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062310#comment-13062310
 ] 

Scott Dworkis commented on CASSANDRA-2874:
--

i actually don't see any messages for that particular log number.  on 
other nodes i see lines that appear related like this:

/data/var/log/cassandra/system.log.47:DEBUG [COMMIT-LOG-WRITER] 2011-06-28 
06:13:56,668 CommitLog.java (line 473) Not safe to delete commit log 
CommitLogSegment(/data/var/lib/cassandra/commitlog/CommitLog-1308613845639.log);
 
dirty is LocationInfo (0), ; hasNext: true

also, i noticed that the .log.header file and the .log file have different 
numbers... does that seem wrong?

CommitLog-1308192929288.log.header  CommitLog-1309995117622.log

-scott





 commitlogs are not draining
 ---

 Key: CASSANDRA-2874
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2874
 Project: Cassandra
  Issue Type: Bug
  Components: Tools
Affects Versions: 0.8.0
 Environment: nodetool
Reporter: Scott Dworkis
Priority: Minor
 Attachments: CommitLog-1308192929288.log.header


 i have commitlogs on every node in the ring that will not drain after 
 invoking a drain with nodetool.  i'll attach one of the log headers at the 
 request of Aaron Morton.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2870) dynamic snitch + read repair off can cause LOCAL_QUORUM reads to return spurious UnavailableException

2011-07-08 Thread Jeremy Hanna (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2870?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13062324#comment-13062324
 ] 

Jeremy Hanna commented on CASSANDRA-2870:
-

Okay - it just seemed like a higher priority issue with the scope expanded.  
We'll probably just disable dynamic snitch until the fix is in a release then.

 dynamic snitch + read repair off can cause LOCAL_QUORUM reads to return 
 spurious UnavailableException
 -

 Key: CASSANDRA-2870
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2870
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.0
Reporter: Jonathan Ellis
Assignee: Jonathan Ellis
Priority: Minor
 Fix For: 0.7.8, 0.8.2

 Attachments: 2870.txt


 When Read Repair is off, we want to avoid doing requests to more nodes than 
 necessary to satisfy the ConsistencyLevel.  ReadCallback does this here:
 {code}
 this.endpoints = repair || resolver instanceof RowRepairResolver
? endpoints
: endpoints.subList(0, Math.min(endpoints.size(), 
 blockfor)); // min so as to not throw exception until assureSufficient is 
 called
 {code}
 You can see that it is assuming that the endpoints list is sorted in order 
 of preferred-ness for the read.
 Then the LOCAL_QUORUM code in DatacenterReadCallback checks to see if we have 
 enough nodes to do the read:
 {code}
 int localEndpoints = 0;
 for (InetAddress endpoint : endpoints)
 {
 if (localdc.equals(snitch.getDatacenter(endpoint)))
 localEndpoints++;
 }
 if (localEndpoints  blockfor)
 throw new UnavailableException();
 {code}
 So if repair is off (so we truncate our endpoints list) AND dynamic snitch 
 has decided that nodes in another DC are to be preferred over local ones, 
 we'll throw UE even if all the replicas are healthy.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira