[jira] Created: (CASSANDRA-1904) SSTable doesn't handle corrupt (empty) tmp files
SSTable doesn't handle corrupt (empty) tmp files Key: CASSANDRA-1904 URL: https://issues.apache.org/jira/browse/CASSANDRA-1904 Project: Cassandra Issue Type: Bug Affects Versions: 0.7.0 rc 2 Reporter: Timo Nentwig Fix For: 0.7.0 Applies to 0.7rc3 as well, but not yet selectable in Jira. cassandra stumbles upons empty Data files and crashes during startup rather than ignoring these files: java.lang.ArithmeticException: / by zero at org.apache.cassandra.io.sstable.SSTable.estimateRowsFromIndex(SSTable.java:233) at org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:284) at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:200) at org.apache.cassandra.db.ColumnFamilyStore.init(ColumnFamilyStore.java:225) at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:448) at org.apache.cassandra.db.ColumnFamilyStore.addIndex(ColumnFamilyStore.java:305) at org.apache.cassandra.db.ColumnFamilyStore.init(ColumnFamilyStore.java:246) at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:448) at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:436) at org.apache.cassandra.db.Table.initCf(Table.java:360) at org.apache.cassandra.db.Table.init(Table.java:290) at org.apache.cassandra.db.Table.open(Table.java:107) at org.apache.cassandra.service.AbstractCassandraDaemon.setup(AbstractCassandraDaemon.java:138) at org.apache.cassandra.thrift.CassandraDaemon.setup(CassandraDaemon.java:55) at org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:216) at org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:134) Exception encountered during startup. java.lang.ArithmeticException: / by zero at org.apache.cassandra.io.sstable.SSTable.estimateRowsFromIndex(SSTable.java:233) at org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:284) at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:200) at org.apache.cassandra.db.ColumnFamilyStore.init(ColumnFamilyStore.java:225) at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:448) at org.apache.cassandra.db.ColumnFamilyStore.addIndex(ColumnFamilyStore.java:305) at org.apache.cassandra.db.ColumnFamilyStore.init(ColumnFamilyStore.java:246) at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:448) at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:436) at org.apache.cassandra.db.Table.initCf(Table.java:360) at org.apache.cassandra.db.Table.init(Table.java:290) at org.apache.cassandra.db.Table.open(Table.java:107) at org.apache.cassandra.service.AbstractCassandraDaemon.setup(AbstractCassandraDaemon.java:138) at org.apache.cassandra.thrift.CassandraDaemon.setup(CassandraDaemon.java:55) at org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:216) at org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:134) The empty Data/Index tmp files were in my case created and left over when I attempted to create a secondary index at runtime which crashed the JVM due to OOM. SSTable handles IOExceptions so it should be an easy fix: in SSTable.estimateRowsFromIndex() just check for ifile.length() ==ifile.getFilePointer()==keys==0 and throw an IOException. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Assigned: (CASSANDRA-959) Allow different timeouts for different classes of operation
[ https://issues.apache.org/jira/browse/CASSANDRA-959?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis reassigned CASSANDRA-959: Assignee: Pavel Yaskevich (was: T Jake Luciani) Allow different timeouts for different classes of operation --- Key: CASSANDRA-959 URL: https://issues.apache.org/jira/browse/CASSANDRA-959 Project: Cassandra Issue Type: New Feature Components: Core Reporter: Jonathan Ellis Assignee: Pavel Yaskevich Priority: Minor Fix For: 0.8 Currently we have one rpc timeout for intra-node operations. But applying the same timeout to read one row, read multiple rows [by key], and range query multiple rows feels like an increasingly uncomfortable fit. (See e.g. CASSANDRA-919.) -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-809) Full disk can result in being marked down
[ https://issues.apache.org/jira/browse/CASSANDRA-809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975274#action_12975274 ] Jonathan Ellis commented on CASSANDRA-809: -- Updating for the past 10 months' worth of changes: bq. Our node that hit this condition is essentially dead (its not gossiping or accepting any writes or reads, but is still alive). This is basically fixed now that flow control is implemented (CASSANDRA-685) and refined (CASSANDRA-1358). bq. It appears that there are n threads for n data directories that we flush to, but they're not dedicated to a data directory. We should have a thread per data directory and have that thread dedicated to that directory At least until we cap sstable size (CASSANDRA-1608?), one data volume is going to be the recommended configuration, so this is low priority. bq. if a disk fills up, we stop trying to write to it bq. if we're about to write more data to a disk than space available, we don't try and write to that disk these two Cassandra has always done on compaction. less sure about flush. the nice thing about writes is that erroring out is almost identical to being completely down for ConsistencyLevel purposes. bq. we balance data relatively evenly between disks also low priority given the above. bq. if a disk is misbehaving for a period of time, we stop using it and assume that data is lost (potentially notify an operator as well) this is the biggest problem right now: if a disk/volume goes down, the rest of the node (in particular gossip) will keep functioning, so other nodes will continue trying to read from it. short term the best fix for this is to provide timeout information to the dynamic snitch (CASSANDRA-1905) so it can route around such nodes. Full disk can result in being marked down - Key: CASSANDRA-809 URL: https://issues.apache.org/jira/browse/CASSANDRA-809 Project: Cassandra Issue Type: Bug Reporter: Ryan King Fix For: 0.8 We had a node file up the disk under one of two data directories. The result was that the node stopped making progress. The problem appears to be this (I'll update with more details as we find them): When new tasks are put onto most queues in Cassandra, if there isn't a thread in the pool to handle the task immediately, the task in run in the caller's thread (org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor:69 sets the caller-runs policy). The queue in question here is the queue that manages flushes, which is enqueued to from various places in our code (and therefore likely from multiple threads). Assuming that the full disk meant that no threads doing flushing could make progress (it appears that way) eventually any thread that calls the flush code would become stalled. Assuming our analysis is right (and we're still looking into it) we need to make a change. Here's a proposal so far: SHORT TERM: * change the TheadPoolExecutor policy to not be caller runs. This will let other threads make progress in the event that one pool is stalled LONG TERM * It appears that there are n threads for n data directories that we flush to, but they're not dedicated to a data directory. We should have a thread per data directory and have that thread dedicated to that directory * Perhaps we could use the failure detector on disks? -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1311) Support (asynchronous) triggers
[ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975280#action_12975280 ] Jonathan Ellis commented on CASSANDRA-1311: --- i take it this is best-suited for triggers that are - invoked frequently - idempotent so that missing an update once in a while (because of a server restart after row update but before trigger processing) is not a big deal? what use cases do we have for this? indexing actually does not fit this description, since if you miss updating an index row for changing the value from 4 to 6, firing the trigger for the change from 6 to 8 will not fix the invalid index entry for 4. Support (asynchronous) triggers --- Key: CASSANDRA-1311 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311 Project: Cassandra Issue Type: New Feature Components: Contrib Reporter: Maxim Grinev Fix For: 0.8 Attachments: HOWTO-PatchAndRunTriggerExample-update1.txt, HOWTO-PatchAndRunTriggerExample.txt, ImplementationDetails-update1.pdf, ImplementationDetails.pdf, trunk-967053.txt, trunk-984391-update1.txt, trunk-984391-update2.txt Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation. Please find the motivation, triggers description and list of applications: http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/ An example of using triggers for indexing: http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/ Implementation details are attached. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1822) Row level coverage in LegacySSTableTest
[ https://issues.apache.org/jira/browse/CASSANDRA-1822?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975286#action_12975286 ] Gary Dusbabek commented on CASSANDRA-1822: -- * I notice that after 0002 LegacySSTableTest fails and the tables themselves are replaced later on. How come? * In LegacySSTableTest the BRAF reference (line 102) can be removed. Row level coverage in LegacySSTableTest --- Key: CASSANDRA-1822 URL: https://issues.apache.org/jira/browse/CASSANDRA-1822 Project: Cassandra Issue Type: Improvement Reporter: Stu Hood Assignee: Stu Hood Priority: Minor Fix For: 0.7.1 Attachments: 1822.tgz, legacy-sstables.tgz LegacySSTableTest should check compatibility of content within rows. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
svn commit: r1053105 - in /cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra: cache/ICacheExpungeHook.java service/ConsistencyChecker.java utils/ExpiringMap.java
Author: jbellis Date: Mon Dec 27 16:48:16 2010 New Revision: 1053105 URL: http://svn.apache.org/viewvc?rev=1053105view=rev Log: convert ConsistencyChecker to use an executor as in 0.7 to ease merging of CASSANDRA-1905 patch by jbellis Removed: cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/cache/ICacheExpungeHook.java Modified: cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/ConsistencyChecker.java cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/utils/ExpiringMap.java Modified: cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/ConsistencyChecker.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/ConsistencyChecker.java?rev=1053105r1=1053104r2=1053105view=diff == --- cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/ConsistencyChecker.java (original) +++ cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/ConsistencyChecker.java Mon Dec 27 16:48:16 2010 @@ -26,12 +26,15 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; -import org.apache.log4j.Logger; import org.apache.commons.lang.ArrayUtils; import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import org.apache.cassandra.cache.ICacheExpungeHook; import org.apache.cassandra.concurrent.StageManager; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ColumnFamily; @@ -41,7 +44,7 @@ import org.apache.cassandra.db.Row; import org.apache.cassandra.net.IAsyncCallback; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.utils.ExpiringMap; +import org.apache.cassandra.utils.WrappedRunnable; /** * ConsistencyChecker does the following: @@ -59,8 +62,9 @@ import org.apache.cassandra.utils.Expiri */ class ConsistencyChecker implements Runnable { - private static Logger logger_ = Logger.getLogger(ConsistencyChecker.class); -private static ExpiringMapString, String readRepairTable_ = new ExpiringMapString, String(DatabaseDescriptor.getRpcTimeout()); +private static Logger logger_ = LoggerFactory.getLogger(ConsistencyChecker.class); + +private static ScheduledExecutorService executor_ = new ScheduledThreadPoolExecutor(1); // TODO add JMX private final Row row_; protected final ListInetAddress replicas_; @@ -126,7 +130,7 @@ class ConsistencyChecker implements Runn ReadCommand readCommand = constructReadMessage(false); Message message = readCommand.makeReadMessage(); if (logger_.isDebugEnabled()) -logger_.debug(Digest mismatch; re-reading + readCommand_.key + from + message.getMessageId() + @[ + StringUtils.join(replicas_, , ) + ]); +logger_.debug(Digest mismatch; re-reading + readCommand_.key + from + message.getMessageId() + @[ + StringUtils.join(replicas_, , ) + ]); MessagingService.instance.addCallback(new DataRepairHandler(), message.getMessageId()); for (InetAddress endpoint : replicas_) { @@ -144,12 +148,12 @@ class ConsistencyChecker implements Runn } } -class DataRepairHandler implements IAsyncCallback, ICacheExpungeHookString, String +class DataRepairHandler implements IAsyncCallback { private final CollectionMessage responses_ = new LinkedBlockingQueueMessage(); private final ReadResponseResolver readResponseResolver_; private final int majority_; - + public DataRepairHandler() throws IOException { readResponseResolver_ = new ReadResponseResolver(readCommand_.table, readCommand_.key, replicas_.size()); @@ -170,20 +174,15 @@ class ConsistencyChecker implements Runn readResponseResolver_.preprocess(message); if (responses_.size() == majority_) { -String messageId = message.getMessageId(); -readRepairTable_.put(messageId, messageId, this); -} -} - - public void callMe(String key, String value) - { -try - { - readResponseResolver_.resolve(responses_); -} -catch (Exception ex) -{ -throw new RuntimeException(ex); +Runnable runnable = new WrappedRunnable() +{ +
[jira] Commented: (CASSANDRA-1709) CQL keyspace and column family management
[ https://issues.apache.org/jira/browse/CASSANDRA-1709?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975287#action_12975287 ] Gary Dusbabek commented on CASSANDRA-1709: -- * would there be a way to specify indexes at the time of CF creation? I'm fine without it, but somehow it makes me think the grammar is incomplete. * I'd prefer something more descriptive than ADD to specify adding an index. ADD INDEX maybe? I suppose it depends on if we foresee the need to 'add' things other than indexes to a column family in the future (unicorns maybe). CQL keyspace and column family management - Key: CASSANDRA-1709 URL: https://issues.apache.org/jira/browse/CASSANDRA-1709 Project: Cassandra Issue Type: Sub-task Components: API Affects Versions: 0.8 Reporter: Eric Evans Priority: Minor Fix For: 0.8 Original Estimate: 0h Remaining Estimate: 0h CQL specification and implementation for schema management. This corresponds to the following RPC methods: * system_add_column_family() * system_add_keyspace() * system_drop_keyspace() * system_update_keyspace() * system_update_columnfamily() -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1905) count timeouts towards dynamicsnitch latencies
[ https://issues.apache.org/jira/browse/CASSANDRA-1905?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975290#action_12975290 ] Hudson commented on CASSANDRA-1905: --- Integrated in Cassandra-0.6 #34 (See [https://hudson.apache.org/hudson/job/Cassandra-0.6/34/]) convert ConsistencyChecker to use an executor as in 0.7 to ease merging of CASSANDRA-1905 patch by jbellis count timeouts towards dynamicsnitch latencies -- Key: CASSANDRA-1905 URL: https://issues.apache.org/jira/browse/CASSANDRA-1905 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.6.6 Reporter: Jonathan Ellis Assignee: Jonathan Ellis Fix For: 0.6.9, 0.7.1 receiveTiming is only called by ResponseVerbHandler; we need to add timing information for timed-out requests as well. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
svn commit: r1053113 - /cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/FBUtilities.java
Author: jbellis Date: Mon Dec 27 17:24:28 2010 New Revision: 1053113 URL: http://svn.apache.org/viewvc?rev=1053113view=rev Log: replace ad-hoc toString(Map) w/ guava MapJoiner patch by jbellis Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/FBUtilities.java Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/FBUtilities.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/FBUtilities.java?rev=1053113r1=1053112r2=1053113view=diff == --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/FBUtilities.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/FBUtilities.java Mon Dec 27 17:24:28 2010 @@ -36,6 +36,7 @@ import java.util.concurrent.atomic.Atomi import java.util.concurrent.atomic.AtomicLong; import com.google.common.base.Charsets; +import com.google.common.base.Joiner; import org.apache.commons.collections.iterators.CollatingIterator; import org.apache.commons.lang.ArrayUtils; import org.slf4j.Logger; @@ -643,14 +644,8 @@ public class FBUtilities public static String toString(Map?,? map) { -// wtf, why isn't something like this in guava or commons collections? -StringBuilder sb = new StringBuilder({); -for (Map.Entry?,? entry : map.entrySet()) -{ -sb.append(toString(entry.getKey())).append(: ).append(toString(entry.getValue())).append(, ); -} -sb.append(}); -return sb.toString(); +Joiner.MapJoiner joiner = Joiner.on(,).withKeyValueSeparator(:); +return joiner.join(map); } /** slow! */
[jira] Updated: (CASSANDRA-1374) Make snitches configurable at runtime
[ https://issues.apache.org/jira/browse/CASSANDRA-1374?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jon Hermes updated CASSANDRA-1374: -- Attachment: 1374-rebase.txt Just needed to shift the MBean abs. method down a line. Silly diff. Make snitches configurable at runtime - Key: CASSANDRA-1374 URL: https://issues.apache.org/jira/browse/CASSANDRA-1374 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 0.7.0 rc 1 Reporter: Jeremy Hanna Assignee: Jon Hermes Fix For: 0.7.1 Attachments: 1374-2.txt, 1374-rebase.txt, 1374.txt There needs to be the capability to configure snitches at runtime, even though there is now a dynamic endpoint snitch - CASSANDRA-981. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1892) heisenbug in SSTableExportTest
[ https://issues.apache.org/jira/browse/CASSANDRA-1892?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975297#action_12975297 ] Hudson commented on CASSANDRA-1892: --- Integrated in Cassandra-0.7 #118 (See [https://hudson.apache.org/hudson/job/Cassandra-0.7/118/]) heisenbug in SSTableExportTest -- Key: CASSANDRA-1892 URL: https://issues.apache.org/jira/browse/CASSANDRA-1892 Project: Cassandra Issue Type: Bug Reporter: Jonathan Ellis Assignee: T Jake Luciani Priority: Minor Fix For: 0.7.1 Attachments: 1892.txt {code} [junit] java.lang.IndexOutOfBoundsException: Index: 4, Size: 4 [junit] at java.util.ArrayList.RangeCheck(ArrayList.java:547) [junit] at java.util.ArrayList.get(ArrayList.java:322) [junit] at org.apache.cassandra.tools.SSTableExportTest.testExportSimpleCf(SSTableExportTest.java:130) {code} -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
svn commit: r1053117 - /cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/FBUtilities.java
Author: jbellis Date: Mon Dec 27 17:51:33 2010 New Revision: 1053117 URL: http://svn.apache.org/viewvc?rev=1053117view=rev Log: clean up FBUtilities patch by Jon Hermes and jbellis Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/FBUtilities.java Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/FBUtilities.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/FBUtilities.java?rev=1053117r1=1053116r2=1053117view=diff == --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/FBUtilities.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/FBUtilities.java Mon Dec 27 17:51:33 2010 @@ -19,7 +19,6 @@ package org.apache.cassandra.utils; import java.io.*; -import java.lang.reflect.Constructor; import java.lang.reflect.Field; import java.lang.reflect.InvocationTargetException; import java.math.BigInteger; @@ -147,7 +146,7 @@ public class FBUtilities remainder = distance.testBit(0); midpoint = distance.shiftRight(1).add(left).mod(max); } -return new Pair(midpoint, remainder); +return new PairBigInteger, Boolean(midpoint, remainder); } public static ByteBuffer toByteBuffer(int i) @@ -489,9 +488,7 @@ public class FBUtilities public static String decodeToUTF8(ByteBuffer bytes) throws CharacterCodingException { -bytes = bytes.duplicate(); -String decoded = Charsets.UTF_8.newDecoder().decode(bytes).toString(); -return decoded; +return Charsets.UTF_8.newDecoder().decode(bytes.duplicate()).toString(); } public static ByteBuffer toByteBuffer(long n) @@ -558,7 +555,7 @@ public class FBUtilities { if (!partitionerClassName.contains(.)) partitionerClassName = org.apache.cassandra.dht. + partitionerClassName; -return FBUtilities.IPartitionerconstruct(partitionerClassName, partitioner); +return FBUtilities.construct(partitionerClassName, partitioner); } public static AbstractType getComparator(String compareWith) throws ConfigurationException @@ -611,11 +608,10 @@ public class FBUtilities */ public static T T construct(String classname, String readable) throws ConfigurationException { -ClassT cls = FBUtilities.TclassForName(classname, readable); +ClassT cls = FBUtilities.classForName(classname, readable); try { -Constructor ctor = cls.getConstructor(); -return (T)ctor.newInstance(); +return cls.getConstructor().newInstance(); } catch (NoSuchMethodException e) { @@ -648,12 +644,6 @@ public class FBUtilities return joiner.join(map); } -/** slow! */ -private static Object toString(Object o) -{ -return o.getClass().isArray() ? Arrays.toString((Object[]) o) : o.toString(); -} - /** * Used to get access to protected/private field of the specified class * @param klass - name of the class
[jira] Commented: (CASSANDRA-1896) Improve throughput by adding buffering to the inter-node TCP communication
[ https://issues.apache.org/jira/browse/CASSANDRA-1896?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975302#action_12975302 ] Brandon Williams commented on CASSANDRA-1896: - I too see up to a 30% improvement here, with an average of about 17% over many millions of rows. However, with most of the system CPU time now removed, I see ever lower utilization. If I discount the JVM threads with increased priority (from UseThreadPriorities) such as garbage collection, it's ridiculously low - a few percent. So this improves performance, but there's a new mystery, what is bottlenecking us now? Improve throughput by adding buffering to the inter-node TCP communication -- Key: CASSANDRA-1896 URL: https://issues.apache.org/jira/browse/CASSANDRA-1896 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 0.6 Reporter: Tsiki Assignee: Brandon Williams Fix For: 0.6.9, 0.7.1 Attachments: 1896.txt The inbound and outbound TCP implementation under org.apache.cassandra.net does not buffer the socket streams. A simple change in IncomingTcpConnection and OutboundTcpConnection may give a rather big throughput increase. In my tests, I got up t o 30% more out of my cluster. Below is the diff of these two files with buffering included. The diff is over release 0.6.5 but can be quite simply applied also to 0.7. I suggest perhaps to limit the buffered input stream I added in IncomingTcpConnection to 4K. The Outbound implementation can surely be implemented a bit better (remove some of the code I duplicated there). diff -r apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/IncomingTcpConnection.java fix/apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/IncomingTcpConnection.java 44c44 input = new DataInputStream(new BufferedInputStream(socket.getInputStream())); --- input = new DataInputStream(socket.getInputStream()); diff -r apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/OutboundTcpConnection.java fix/apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/OutboundTcpConnection.java 77d76 byte[] buf = new byte[4096]; 80,112c79,89 int l = 0; ByteBuffer bb; while ((bb = queue.peek()) != null l+bb.limit() buf.length) { bb = take(); System.arraycopy(bb.array(), 0, buf, l, bb.limit()); l += bb.limit(); } if (l == 0) { bb = take(); if (bb == CLOSE_SENTINEL) { disconnect(); continue; } if (socket != null || connect()) writeConnected(bb); else // clear out the queue, else gossip messages back up. queue.clear(); } else { if (socket != null || connect()) { try { output.write(buf, 0, l); if (queue.peek() == null) output.flush(); } catch (IOException e) { logger.info(error writing to + endpoint); disconnect(); } } else { queue.clear(); } } --- ByteBuffer bb = take(); if (bb == CLOSE_SENTINEL) { disconnect(); continue; } if (socket != null || connect()) writeConnected(bb); else // clear out the queue, else gossip messages back up. queue.clear(); -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1896) Improve throughput by adding buffering to the inter-node TCP communication
[ https://issues.apache.org/jira/browse/CASSANDRA-1896?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975306#action_12975306 ] Brandon Williams commented on CASSANDRA-1896: - I should note that the 'few percent' situation was with the commitlog hacked out to be disabled, otherwise the CL writer thread is at about 15% - still very low utilization overall. Improve throughput by adding buffering to the inter-node TCP communication -- Key: CASSANDRA-1896 URL: https://issues.apache.org/jira/browse/CASSANDRA-1896 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 0.6 Reporter: Tsiki Assignee: Brandon Williams Fix For: 0.6.9, 0.7.1 Attachments: 1896.txt The inbound and outbound TCP implementation under org.apache.cassandra.net does not buffer the socket streams. A simple change in IncomingTcpConnection and OutboundTcpConnection may give a rather big throughput increase. In my tests, I got up t o 30% more out of my cluster. Below is the diff of these two files with buffering included. The diff is over release 0.6.5 but can be quite simply applied also to 0.7. I suggest perhaps to limit the buffered input stream I added in IncomingTcpConnection to 4K. The Outbound implementation can surely be implemented a bit better (remove some of the code I duplicated there). diff -r apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/IncomingTcpConnection.java fix/apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/IncomingTcpConnection.java 44c44 input = new DataInputStream(new BufferedInputStream(socket.getInputStream())); --- input = new DataInputStream(socket.getInputStream()); diff -r apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/OutboundTcpConnection.java fix/apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/OutboundTcpConnection.java 77d76 byte[] buf = new byte[4096]; 80,112c79,89 int l = 0; ByteBuffer bb; while ((bb = queue.peek()) != null l+bb.limit() buf.length) { bb = take(); System.arraycopy(bb.array(), 0, buf, l, bb.limit()); l += bb.limit(); } if (l == 0) { bb = take(); if (bb == CLOSE_SENTINEL) { disconnect(); continue; } if (socket != null || connect()) writeConnected(bb); else // clear out the queue, else gossip messages back up. queue.clear(); } else { if (socket != null || connect()) { try { output.write(buf, 0, l); if (queue.peek() == null) output.flush(); } catch (IOException e) { logger.info(error writing to + endpoint); disconnect(); } } else { queue.clear(); } } --- ByteBuffer bb = take(); if (bb == CLOSE_SENTINEL) { disconnect(); continue; } if (socket != null || connect()) writeConnected(bb); else // clear out the queue, else gossip messages back up. queue.clear(); -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1311) Support (asynchronous) triggers
[ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975311#action_12975311 ] David Erickson commented on CASSANDRA-1311: --- I haven't dug into this implementation of triggers, but the use case we had in mind is using the database as a shared communication bus amongst distributed nodes. If node 1 makes a change to its Cassandra instance, once the data propagates to the other Cassandra instances the triggers can alert other nodes that a change has been made and they need to do some processing. This is much better than polling for changes. Alternatively the nodes would have to have their own protocol to alert each other outside of the database layer that changes have been made. Support (asynchronous) triggers --- Key: CASSANDRA-1311 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311 Project: Cassandra Issue Type: New Feature Components: Contrib Reporter: Maxim Grinev Fix For: 0.8 Attachments: HOWTO-PatchAndRunTriggerExample-update1.txt, HOWTO-PatchAndRunTriggerExample.txt, ImplementationDetails-update1.pdf, ImplementationDetails.pdf, trunk-967053.txt, trunk-984391-update1.txt, trunk-984391-update2.txt Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation. Please find the motivation, triggers description and list of applications: http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/ An example of using triggers for indexing: http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/ Implementation details are attached. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Issue Comment Edited: (CASSANDRA-1311) Support (asynchronous) triggers
[ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975311#action_12975311 ] David Erickson edited comment on CASSANDRA-1311 at 12/27/10 2:00 PM: - I haven't dug into this implementation of triggers, but a use case could be using Cassandra as a shared communication bus amongst distributed nodes. If node 1 makes a change to its Cassandra instance, the data then propagates to the other Cassandra instances, and the triggers alert other nodes that a change has been made and they need to do some processing, which is better than polling for the same changes. Alternatively the nodes would have to have their own protocol to alert each other outside of the database layer that changes have been made. was (Author: derickso): I haven't dug into this implementation of triggers, but the use case we had in mind is using the database as a shared communication bus amongst distributed nodes. If node 1 makes a change to its Cassandra instance, once the data propagates to the other Cassandra instances the triggers can alert other nodes that a change has been made and they need to do some processing. This is much better than polling for changes. Alternatively the nodes would have to have their own protocol to alert each other outside of the database layer that changes have been made. Support (asynchronous) triggers --- Key: CASSANDRA-1311 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311 Project: Cassandra Issue Type: New Feature Components: Contrib Reporter: Maxim Grinev Fix For: 0.8 Attachments: HOWTO-PatchAndRunTriggerExample-update1.txt, HOWTO-PatchAndRunTriggerExample.txt, ImplementationDetails-update1.pdf, ImplementationDetails.pdf, trunk-967053.txt, trunk-984391-update1.txt, trunk-984391-update2.txt Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation. Please find the motivation, triggers description and list of applications: http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/ An example of using triggers for indexing: http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/ Implementation details are attached. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1896) Improve throughput by adding buffering to the inter-node TCP communication
[ https://issues.apache.org/jira/browse/CASSANDRA-1896?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975315#action_12975315 ] Tsiki commented on CASSANDRA-1896: -- I am not sure if this is related to what you see or not but it may be: As I wrote before, with my tests, after adding this I got better throughput. For the issue of utilization, I had to play with memory parameters. Before doing so, I had young generation GCs happen very too often and a lot of CPU was spent on the GCs themselves and on stopping and restarting the real threads. In that scenario I had several young generation GCs every second. After changing memory parameters, I had one GC every several seconds which completely solved it for me. Improve throughput by adding buffering to the inter-node TCP communication -- Key: CASSANDRA-1896 URL: https://issues.apache.org/jira/browse/CASSANDRA-1896 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 0.6 Reporter: Tsiki Assignee: Brandon Williams Fix For: 0.6.9, 0.7.1 Attachments: 1896.txt The inbound and outbound TCP implementation under org.apache.cassandra.net does not buffer the socket streams. A simple change in IncomingTcpConnection and OutboundTcpConnection may give a rather big throughput increase. In my tests, I got up t o 30% more out of my cluster. Below is the diff of these two files with buffering included. The diff is over release 0.6.5 but can be quite simply applied also to 0.7. I suggest perhaps to limit the buffered input stream I added in IncomingTcpConnection to 4K. The Outbound implementation can surely be implemented a bit better (remove some of the code I duplicated there). diff -r apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/IncomingTcpConnection.java fix/apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/IncomingTcpConnection.java 44c44 input = new DataInputStream(new BufferedInputStream(socket.getInputStream())); --- input = new DataInputStream(socket.getInputStream()); diff -r apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/OutboundTcpConnection.java fix/apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/OutboundTcpConnection.java 77d76 byte[] buf = new byte[4096]; 80,112c79,89 int l = 0; ByteBuffer bb; while ((bb = queue.peek()) != null l+bb.limit() buf.length) { bb = take(); System.arraycopy(bb.array(), 0, buf, l, bb.limit()); l += bb.limit(); } if (l == 0) { bb = take(); if (bb == CLOSE_SENTINEL) { disconnect(); continue; } if (socket != null || connect()) writeConnected(bb); else // clear out the queue, else gossip messages back up. queue.clear(); } else { if (socket != null || connect()) { try { output.write(buf, 0, l); if (queue.peek() == null) output.flush(); } catch (IOException e) { logger.info(error writing to + endpoint); disconnect(); } } else { queue.clear(); } } --- ByteBuffer bb = take(); if (bb == CLOSE_SENTINEL) { disconnect(); continue; } if (socket != null || connect()) writeConnected(bb); else // clear out the queue, else gossip messages back up. queue.clear(); -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1896) Improve throughput by adding buffering to the inter-node TCP communication
[ https://issues.apache.org/jira/browse/CASSANDRA-1896?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975319#action_12975319 ] Tsiki commented on CASSANDRA-1896: -- As I had this issue not only with Cassandra, I tried several settings. First, was just to increase the heap size (Xmx). Then I tried specifying XX:NewRatio but that didn't work well at all. I also tried setting Xmx=Xms with NewRatio. Finally, I landed with XX:NewSize to explicitly set the young generation size. I didn't play around with survivor spaces. Improve throughput by adding buffering to the inter-node TCP communication -- Key: CASSANDRA-1896 URL: https://issues.apache.org/jira/browse/CASSANDRA-1896 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 0.6 Reporter: Tsiki Assignee: Brandon Williams Fix For: 0.6.9, 0.7.1 Attachments: 1896.txt The inbound and outbound TCP implementation under org.apache.cassandra.net does not buffer the socket streams. A simple change in IncomingTcpConnection and OutboundTcpConnection may give a rather big throughput increase. In my tests, I got up t o 30% more out of my cluster. Below is the diff of these two files with buffering included. The diff is over release 0.6.5 but can be quite simply applied also to 0.7. I suggest perhaps to limit the buffered input stream I added in IncomingTcpConnection to 4K. The Outbound implementation can surely be implemented a bit better (remove some of the code I duplicated there). diff -r apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/IncomingTcpConnection.java fix/apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/IncomingTcpConnection.java 44c44 input = new DataInputStream(new BufferedInputStream(socket.getInputStream())); --- input = new DataInputStream(socket.getInputStream()); diff -r apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/OutboundTcpConnection.java fix/apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/OutboundTcpConnection.java 77d76 byte[] buf = new byte[4096]; 80,112c79,89 int l = 0; ByteBuffer bb; while ((bb = queue.peek()) != null l+bb.limit() buf.length) { bb = take(); System.arraycopy(bb.array(), 0, buf, l, bb.limit()); l += bb.limit(); } if (l == 0) { bb = take(); if (bb == CLOSE_SENTINEL) { disconnect(); continue; } if (socket != null || connect()) writeConnected(bb); else // clear out the queue, else gossip messages back up. queue.clear(); } else { if (socket != null || connect()) { try { output.write(buf, 0, l); if (queue.peek() == null) output.flush(); } catch (IOException e) { logger.info(error writing to + endpoint); disconnect(); } } else { queue.clear(); } } --- ByteBuffer bb = take(); if (bb == CLOSE_SENTINEL) { disconnect(); continue; } if (socket != null || connect()) writeConnected(bb); else // clear out the queue, else gossip messages back up. queue.clear(); -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
svn commit: r1053169 - /cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/MessagingService.java
Author: jbellis Date: Mon Dec 27 20:16:07 2010 New Revision: 1053169 URL: http://svn.apache.org/viewvc?rev=1053169view=rev Log: rename messageId parameters Modified: cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/MessagingService.java Modified: cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/MessagingService.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/MessagingService.java?rev=1053169r1=1053168r2=1053169view=diff == --- cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/MessagingService.java (original) +++ cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/MessagingService.java Mon Dec 27 20:16:07 2010 @@ -396,29 +396,29 @@ public class MessagingService } } -public static IAsyncCallback getRegisteredCallback(String key) +public static IAsyncCallback getRegisteredCallback(String messageId) { -return callbackMap_.get(key); +return callbackMap_.get(messageId); } -public static void removeRegisteredCallback(String key) +public static void removeRegisteredCallback(String messageId) { -callbackMap_.remove(key); +callbackMap_.remove(messageId); } -public static IAsyncResult getAsyncResult(String key) +public static IAsyncResult getAsyncResult(String messageId) { -return taskCompletionMap_.remove(key); +return taskCompletionMap_.remove(messageId); } -public static long getRegisteredCallbackAge(String key) +public static long getRegisteredCallbackAge(String messageId) { -return callbackMap_.getAge(key); +return callbackMap_.getAge(messageId); } -public static long getAsyncResultAge(String key) +public static long getAsyncResultAge(String messageId) { -return taskCompletionMap_.getAge(key); +return taskCompletionMap_.getAge(messageId); } public static void validateMagic(int magic) throws IOException
[jira] Commented: (CASSANDRA-1108) ability to forcibly mark machines failed
[ https://issues.apache.org/jira/browse/CASSANDRA-1108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975332#action_12975332 ] Brandon Williams commented on CASSANDRA-1108: - I see two approaches here: the more complex one, where we add a new flag to gossip saying this node is blacklisted and propagate it around, and maintain a map of blacklisted nodes that we check in many places, or one where we just tell the bad node to stop gossiping allowing the FD of the other nodes to mark it down. The downside to the latter approach is that if the bad node is so sick (ie, GC death spiral) you can't complete the JMX call, you're still stuck, but hopefully at that point it's in such bad shape gossip doesn't work either. ability to forcibly mark machines failed Key: CASSANDRA-1108 URL: https://issues.apache.org/jira/browse/CASSANDRA-1108 Project: Cassandra Issue Type: New Feature Components: Tools Reporter: Jonathan Ellis Assignee: Brandon Williams Priority: Minor Fix For: 0.7.1 For when a node is failing but not yet so badly that it can't participate in gossip (e.g. hard disk failing but not dead yet) we should give operators the power to forcibly mark a node as dead. I think we'd need to add an extra flag in gossip to say this deadness is operator-imposed or the next heartbeat will flip it back to live. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (CASSANDRA-1904) Crash during startup: SSTable doesn't handle corrupt (empty) tmp files
[ https://issues.apache.org/jira/browse/CASSANDRA-1904?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-1904: -- Fix Version/s: (was: 0.7.0) 0.7.1 Assignee: Gary Dusbabek Crash during startup: SSTable doesn't handle corrupt (empty) tmp files -- Key: CASSANDRA-1904 URL: https://issues.apache.org/jira/browse/CASSANDRA-1904 Project: Cassandra Issue Type: Bug Affects Versions: 0.7.0 rc 2 Reporter: Timo Nentwig Assignee: Gary Dusbabek Fix For: 0.7.1 Applies to 0.7rc3 as well, but not yet selectable in Jira. cassandra stumbles upons empty Data files and crashes during startup rather than ignoring these files: java.lang.ArithmeticException: / by zero at org.apache.cassandra.io.sstable.SSTable.estimateRowsFromIndex(SSTable.java:233) at org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:284) at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:200) at org.apache.cassandra.db.ColumnFamilyStore.init(ColumnFamilyStore.java:225) at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:448) at org.apache.cassandra.db.ColumnFamilyStore.addIndex(ColumnFamilyStore.java:305) at org.apache.cassandra.db.ColumnFamilyStore.init(ColumnFamilyStore.java:246) at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:448) at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:436) at org.apache.cassandra.db.Table.initCf(Table.java:360) at org.apache.cassandra.db.Table.init(Table.java:290) at org.apache.cassandra.db.Table.open(Table.java:107) at org.apache.cassandra.service.AbstractCassandraDaemon.setup(AbstractCassandraDaemon.java:138) at org.apache.cassandra.thrift.CassandraDaemon.setup(CassandraDaemon.java:55) at org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:216) at org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:134) Exception encountered during startup. java.lang.ArithmeticException: / by zero at org.apache.cassandra.io.sstable.SSTable.estimateRowsFromIndex(SSTable.java:233) at org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:284) at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:200) at org.apache.cassandra.db.ColumnFamilyStore.init(ColumnFamilyStore.java:225) at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:448) at org.apache.cassandra.db.ColumnFamilyStore.addIndex(ColumnFamilyStore.java:305) at org.apache.cassandra.db.ColumnFamilyStore.init(ColumnFamilyStore.java:246) at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:448) at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:436) at org.apache.cassandra.db.Table.initCf(Table.java:360) at org.apache.cassandra.db.Table.init(Table.java:290) at org.apache.cassandra.db.Table.open(Table.java:107) at org.apache.cassandra.service.AbstractCassandraDaemon.setup(AbstractCassandraDaemon.java:138) at org.apache.cassandra.thrift.CassandraDaemon.setup(CassandraDaemon.java:55) at org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:216) at org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:134) The empty Data/Index tmp files were in my case created and left over when I attempted to create a secondary index at runtime which crashed the JVM due to OOM. SSTable handles IOExceptions so it should be an easy fix: in SSTable.estimateRowsFromIndex() just check for ifile.length() ==ifile.getFilePointer()==keys==0 and throw an IOException. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1709) CQL keyspace and column family management
[ https://issues.apache.org/jira/browse/CASSANDRA-1709?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975335#action_12975335 ] Eric Evans commented on CASSANDRA-1709: --- bq. would there be a way to specify indexes at the time of CF creation? I'm fine without it, but somehow it makes me think the grammar is incomplete. I couldn't think of any that made sense to me. FWIW, the CLI handles this by using a JSON-like dictionary to specify column metadata (including indexes), which feels wrong to me. So far, we've gone to a lot of trouble to be as SQLish as possible, so it's worth pointing out that you can't create an index and table in one-shot either (and I think this is OK). bq. I'd prefer something more descriptive than ADD to specify adding an index. ADD INDEX maybe? I suppose it depends on if we foresee the need to 'add' things other than indexes to a column family in the future (unicorns maybe). Sticking with the theme of being as SQL-like as reasonable, {{CREATE INDEX ...}} would the way to go, and I almost suggested that. What made me rethink was that we still need a way to set the other metadata (like validation_class), and I wasn't sure it was worth special-casing indexes. CQL keyspace and column family management - Key: CASSANDRA-1709 URL: https://issues.apache.org/jira/browse/CASSANDRA-1709 Project: Cassandra Issue Type: Sub-task Components: API Affects Versions: 0.8 Reporter: Eric Evans Priority: Minor Fix For: 0.8 Original Estimate: 0h Remaining Estimate: 0h CQL specification and implementation for schema management. This corresponds to the following RPC methods: * system_add_column_family() * system_add_keyspace() * system_drop_keyspace() * system_update_keyspace() * system_update_columnfamily() -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1108) ability to forcibly mark machines failed
[ https://issues.apache.org/jira/browse/CASSANDRA-1108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975336#action_12975336 ] Jonathan Ellis commented on CASSANDRA-1108: --- +1 stopping gossip, if it's so sick you can't do that then (a) experience shows it's not going to be gossiping anyway and (b) you're probably going to need to kill -9ing it instead :) ability to forcibly mark machines failed Key: CASSANDRA-1108 URL: https://issues.apache.org/jira/browse/CASSANDRA-1108 Project: Cassandra Issue Type: New Feature Components: Tools Reporter: Jonathan Ellis Assignee: Brandon Williams Priority: Minor Fix For: 0.7.1 For when a node is failing but not yet so badly that it can't participate in gossip (e.g. hard disk failing but not dead yet) we should give operators the power to forcibly mark a node as dead. I think we'd need to add an extra flag in gossip to say this deadness is operator-imposed or the next heartbeat will flip it back to live. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Created: (CASSANDRA-1907) AbstractCassandraDaemon blows up when log4j config is specified using a physical file.
AbstractCassandraDaemon blows up when log4j config is specified using a physical file. -- Key: CASSANDRA-1907 URL: https://issues.apache.org/jira/browse/CASSANDRA-1907 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.7.0 rc 2 Reporter: Gary Dusbabek Assignee: Gary Dusbabek Priority: Minor Fix For: 0.7.1 -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (CASSANDRA-1907) AbstractCassandraDaemon blows up when log4j config is specified using a physical file.
[ https://issues.apache.org/jira/browse/CASSANDRA-1907?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gary Dusbabek updated CASSANDRA-1907: - Attachment: 0001-1907.patch AbstractCassandraDaemon blows up when log4j config is specified using a physical file. -- Key: CASSANDRA-1907 URL: https://issues.apache.org/jira/browse/CASSANDRA-1907 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.7.0 rc 2 Reporter: Gary Dusbabek Assignee: Gary Dusbabek Priority: Minor Fix For: 0.7.1 Attachments: 0001-1907.patch -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (CASSANDRA-1907) AbstractCassandraDaemon blows up when log4j config is specified using a physical file.
[ https://issues.apache.org/jira/browse/CASSANDRA-1907?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-1907: -- Reviewer: tjake AbstractCassandraDaemon blows up when log4j config is specified using a physical file. -- Key: CASSANDRA-1907 URL: https://issues.apache.org/jira/browse/CASSANDRA-1907 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.7.0 rc 2 Reporter: Gary Dusbabek Assignee: Gary Dusbabek Priority: Minor Fix For: 0.7.1 Attachments: 0001-1907.patch -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Issue Comment Edited: (CASSANDRA-1896) Improve throughput by adding buffering to the inter-node TCP communication
[ https://issues.apache.org/jira/browse/CASSANDRA-1896?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975319#action_12975319 ] Tsiki edited comment on CASSANDRA-1896 at 12/27/10 4:23 PM: As I had this issue not only with Cassandra, I tried several settings. First, was just to increase the heap size (Xmx). Then I tried specifying XX:NewRatio but that didn't work well at all. I also tried setting Xmx=Xms with NewRatio. Finally, I landed with XX:NewSize to explicitly set the young generation size. I didn't play around with survivor spaces. By the way. What are you running your test on? (HW / OS / etc) was (Author: tsiki): As I had this issue not only with Cassandra, I tried several settings. First, was just to increase the heap size (Xmx). Then I tried specifying XX:NewRatio but that didn't work well at all. I also tried setting Xmx=Xms with NewRatio. Finally, I landed with XX:NewSize to explicitly set the young generation size. I didn't play around with survivor spaces. Improve throughput by adding buffering to the inter-node TCP communication -- Key: CASSANDRA-1896 URL: https://issues.apache.org/jira/browse/CASSANDRA-1896 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 0.6 Reporter: Tsiki Assignee: Brandon Williams Fix For: 0.6.9, 0.7.1 Attachments: 1896.txt The inbound and outbound TCP implementation under org.apache.cassandra.net does not buffer the socket streams. A simple change in IncomingTcpConnection and OutboundTcpConnection may give a rather big throughput increase. In my tests, I got up t o 30% more out of my cluster. Below is the diff of these two files with buffering included. The diff is over release 0.6.5 but can be quite simply applied also to 0.7. I suggest perhaps to limit the buffered input stream I added in IncomingTcpConnection to 4K. The Outbound implementation can surely be implemented a bit better (remove some of the code I duplicated there). diff -r apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/IncomingTcpConnection.java fix/apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/IncomingTcpConnection.java 44c44 input = new DataInputStream(new BufferedInputStream(socket.getInputStream())); --- input = new DataInputStream(socket.getInputStream()); diff -r apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/OutboundTcpConnection.java fix/apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/OutboundTcpConnection.java 77d76 byte[] buf = new byte[4096]; 80,112c79,89 int l = 0; ByteBuffer bb; while ((bb = queue.peek()) != null l+bb.limit() buf.length) { bb = take(); System.arraycopy(bb.array(), 0, buf, l, bb.limit()); l += bb.limit(); } if (l == 0) { bb = take(); if (bb == CLOSE_SENTINEL) { disconnect(); continue; } if (socket != null || connect()) writeConnected(bb); else // clear out the queue, else gossip messages back up. queue.clear(); } else { if (socket != null || connect()) { try { output.write(buf, 0, l); if (queue.peek() == null) output.flush(); } catch (IOException e) { logger.info(error writing to + endpoint); disconnect(); } } else { queue.clear(); } } --- ByteBuffer bb = take(); if (bb == CLOSE_SENTINEL) { disconnect(); continue; } if (socket != null || connect()) writeConnected(bb); else // clear out the queue, else gossip messages back up. queue.clear(); -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Created: (CASSANDRA-1908) Implement the CLibrary using JNI module to avoid the LGPL dependency on JNA
Implement the CLibrary using JNI module to avoid the LGPL dependency on JNA --- Key: CASSANDRA-1908 URL: https://issues.apache.org/jira/browse/CASSANDRA-1908 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Hiram Chirino Fix For: 0.7.1 Cassandra can't ship JNA out of the box since it's LGPL licensed, so many of the performance optimizing features in the CLibrary class are not available in a simple install. It should be trivial to implement a real JNI library for the CLibrary class. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (CASSANDRA-1908) Implement the CLibrary using JNI module to avoid the LGPL dependency on JNA
[ https://issues.apache.org/jira/browse/CASSANDRA-1908?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Hiram Chirino updated CASSANDRA-1908: - Attachment: cassandra-jni.zip Attaching a port of the latest CLibrary. Build with maven. It code generates the JNI library implementation. See readme.md for more details. Implement the CLibrary using JNI module to avoid the LGPL dependency on JNA --- Key: CASSANDRA-1908 URL: https://issues.apache.org/jira/browse/CASSANDRA-1908 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Hiram Chirino Fix For: 0.7.1 Attachments: cassandra-jni.zip Cassandra can't ship JNA out of the box since it's LGPL licensed, so many of the performance optimizing features in the CLibrary class are not available in a simple install. It should be trivial to implement a real JNI library for the CLibrary class. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1896) Improve throughput by adding buffering to the inter-node TCP communication
[ https://issues.apache.org/jira/browse/CASSANDRA-1896?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975345#action_12975345 ] Brandon Williams commented on CASSANDRA-1896: - Multiple clusters, all running debian lenny. Some real HW (8G ram, 4 cores, separate spindles) and some virtualized (16G ram, 4 cores, shared RAID10). How large was your heap and what did you set XX:NewSize to? Improve throughput by adding buffering to the inter-node TCP communication -- Key: CASSANDRA-1896 URL: https://issues.apache.org/jira/browse/CASSANDRA-1896 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 0.6 Reporter: Tsiki Assignee: Brandon Williams Fix For: 0.6.9, 0.7.1 Attachments: 1896.txt The inbound and outbound TCP implementation under org.apache.cassandra.net does not buffer the socket streams. A simple change in IncomingTcpConnection and OutboundTcpConnection may give a rather big throughput increase. In my tests, I got up t o 30% more out of my cluster. Below is the diff of these two files with buffering included. The diff is over release 0.6.5 but can be quite simply applied also to 0.7. I suggest perhaps to limit the buffered input stream I added in IncomingTcpConnection to 4K. The Outbound implementation can surely be implemented a bit better (remove some of the code I duplicated there). diff -r apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/IncomingTcpConnection.java fix/apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/IncomingTcpConnection.java 44c44 input = new DataInputStream(new BufferedInputStream(socket.getInputStream())); --- input = new DataInputStream(socket.getInputStream()); diff -r apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/OutboundTcpConnection.java fix/apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/OutboundTcpConnection.java 77d76 byte[] buf = new byte[4096]; 80,112c79,89 int l = 0; ByteBuffer bb; while ((bb = queue.peek()) != null l+bb.limit() buf.length) { bb = take(); System.arraycopy(bb.array(), 0, buf, l, bb.limit()); l += bb.limit(); } if (l == 0) { bb = take(); if (bb == CLOSE_SENTINEL) { disconnect(); continue; } if (socket != null || connect()) writeConnected(bb); else // clear out the queue, else gossip messages back up. queue.clear(); } else { if (socket != null || connect()) { try { output.write(buf, 0, l); if (queue.peek() == null) output.flush(); } catch (IOException e) { logger.info(error writing to + endpoint); disconnect(); } } else { queue.clear(); } } --- ByteBuffer bb = take(); if (bb == CLOSE_SENTINEL) { disconnect(); continue; } if (socket != null || connect()) writeConnected(bb); else // clear out the queue, else gossip messages back up. queue.clear(); -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1896) Improve throughput by adding buffering to the inter-node TCP communication
[ https://issues.apache.org/jira/browse/CASSANDRA-1896?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975355#action_12975355 ] Jonathan Ellis commented on CASSANDRA-1896: --- at first glance the ByteBuffer code on write looks buggy, it should be using remaining instead of limit and may not be including offsets correctly. after dealing w/ BB issues we can commit this to 0.6, but for 0.7 I'd prefer to finish CASSANDRA-1788 before muddying the water further. We basically write three ints and a byte array; i wouldn't be surprised if buffering is a win on the ints, but a lose on the byte[]. I.e., this demonstrates that a copy + syscall is faster than four syscalls, but two syscalls [and still no copy] may be faster yet. Improve throughput by adding buffering to the inter-node TCP communication -- Key: CASSANDRA-1896 URL: https://issues.apache.org/jira/browse/CASSANDRA-1896 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 0.6 Reporter: Tsiki Assignee: Brandon Williams Fix For: 0.6.9, 0.7.1 Attachments: 1896.txt The inbound and outbound TCP implementation under org.apache.cassandra.net does not buffer the socket streams. A simple change in IncomingTcpConnection and OutboundTcpConnection may give a rather big throughput increase. In my tests, I got up t o 30% more out of my cluster. Below is the diff of these two files with buffering included. The diff is over release 0.6.5 but can be quite simply applied also to 0.7. I suggest perhaps to limit the buffered input stream I added in IncomingTcpConnection to 4K. The Outbound implementation can surely be implemented a bit better (remove some of the code I duplicated there). diff -r apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/IncomingTcpConnection.java fix/apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/IncomingTcpConnection.java 44c44 input = new DataInputStream(new BufferedInputStream(socket.getInputStream())); --- input = new DataInputStream(socket.getInputStream()); diff -r apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/OutboundTcpConnection.java fix/apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/OutboundTcpConnection.java 77d76 byte[] buf = new byte[4096]; 80,112c79,89 int l = 0; ByteBuffer bb; while ((bb = queue.peek()) != null l+bb.limit() buf.length) { bb = take(); System.arraycopy(bb.array(), 0, buf, l, bb.limit()); l += bb.limit(); } if (l == 0) { bb = take(); if (bb == CLOSE_SENTINEL) { disconnect(); continue; } if (socket != null || connect()) writeConnected(bb); else // clear out the queue, else gossip messages back up. queue.clear(); } else { if (socket != null || connect()) { try { output.write(buf, 0, l); if (queue.peek() == null) output.flush(); } catch (IOException e) { logger.info(error writing to + endpoint); disconnect(); } } else { queue.clear(); } } --- ByteBuffer bb = take(); if (bb == CLOSE_SENTINEL) { disconnect(); continue; } if (socket != null || connect()) writeConnected(bb); else // clear out the queue, else gossip messages back up. queue.clear(); -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
svn commit: r1053205 - in /cassandra/branches/cassandra-0.7: CHANGES.txt src/java/org/apache/cassandra/service/AbstractCassandraDaemon.java
Author: gdusbabek Date: Mon Dec 27 22:13:53 2010 New Revision: 1053205 URL: http://svn.apache.org/viewvc?rev=1053205view=rev Log: log4j configuration wasn't handling configurations specified by URL. patch by Gary Dusbabek, reviewed by Jake Luciani. CASSANDRA-1907 Modified: cassandra/branches/cassandra-0.7/CHANGES.txt cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/AbstractCassandraDaemon.java Modified: cassandra/branches/cassandra-0.7/CHANGES.txt URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/CHANGES.txt?rev=1053205r1=1053204r2=1053205view=diff == --- cassandra/branches/cassandra-0.7/CHANGES.txt (original) +++ cassandra/branches/cassandra-0.7/CHANGES.txt Mon Dec 27 22:13:53 2010 @@ -9,6 +9,7 @@ dev * avoid polluting page cache with commitlog or sstable writes and seq scan operations (CASSANDRA-1470) * add OpenBitSet to support larger bloom filters (CASSANDRA-1555) + * handle URL-specified log4j regression (CASSANDRA-1907) 0.7.0-rc3 Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/AbstractCassandraDaemon.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/AbstractCassandraDaemon.java?rev=1053205r1=1053204r2=1053205view=diff == --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/AbstractCassandraDaemon.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/AbstractCassandraDaemon.java Mon Dec 27 22:13:53 2010 @@ -21,6 +21,8 @@ package org.apache.cassandra.service; import java.io.File; import java.io.IOException; import java.net.InetAddress; +import java.net.MalformedURLException; +import java.net.URL; import java.util.UUID; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.SynchronousQueue; @@ -59,7 +61,20 @@ public abstract class AbstractCassandraD static { String config = System.getProperty(log4j.configuration, log4j-server.properties); - PropertyConfigurator.configureAndWatch(ClassLoader.getSystemResource(config).getFile(), 1); +URL configLocation = null; +try +{ +// try loading from a physical location first. +configLocation = new URL(config); +} +catch (MalformedURLException ex) +{ +// load from the classpath. +configLocation = AbstractCassandraDaemon.class.getClassLoader().getResource(config); +if (configLocation == null) +throw new RuntimeException(Couldn't figure out log4j configuration.); +} +PropertyConfigurator.configureAndWatch(configLocation.getFile(), 1); org.apache.log4j.Logger.getLogger(AbstractCassandraDaemon.class).info(Logging initialized); }
svn commit: r1053207 - in /cassandra/trunk: ./ interface/thrift/gen-java/org/apache/cassandra/thrift/ src/java/org/apache/cassandra/service/ src/java/org/apache/cassandra/utils/ test/unit/org/apache/c
Author: gdusbabek Date: Mon Dec 27 22:19:01 2010 New Revision: 1053207 URL: http://svn.apache.org/viewvc?rev=1053207view=rev Log: merge from 0.7 Modified: cassandra/trunk/ (props changed) cassandra/trunk/CHANGES.txt 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/service/AbstractCassandraDaemon.java cassandra/trunk/src/java/org/apache/cassandra/utils/FBUtilities.java cassandra/trunk/test/unit/org/apache/cassandra/tools/SSTableExportTest.java Propchange: cassandra/trunk/ -- --- svn:mergeinfo (original) +++ svn:mergeinfo Mon Dec 27 22:19:01 2010 @@ -1,5 +1,5 @@ /cassandra/branches/cassandra-0.6:922689-1051640,1051662 -/cassandra/branches/cassandra-0.7:1026517-1052542 +/cassandra/branches/cassandra-0.7:1026517-1053205 /incubator/cassandra/branches/cassandra-0.3:774578-796573 /incubator/cassandra/branches/cassandra-0.4:810145-834239,834349-834350 /incubator/cassandra/branches/cassandra-0.5:72-915439 Modified: cassandra/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/cassandra/trunk/CHANGES.txt?rev=1053207r1=1053206r2=1053207view=diff == --- cassandra/trunk/CHANGES.txt (original) +++ cassandra/trunk/CHANGES.txt Mon Dec 27 22:19:01 2010 @@ -14,6 +14,7 @@ * avoid polluting page cache with commitlog or sstable writes and seq scan operations (CASSANDRA-1470) * add OpenBitSet to support larger bloom filters (CASSANDRA-1555) + * handle URL-specified log4j regression (CASSANDRA-1907) 0.7.0-rc3 Propchange: cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java -- --- svn:mergeinfo (original) +++ svn:mergeinfo Mon Dec 27 22:19:01 2010 @@ -1,5 +1,5 @@ /cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:922689-1051640,1051662 -/cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:1026517-1052542 +/cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:1026517-1053205 /incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/Cassandra.java:774578-796573 /incubator/cassandra/branches/cassandra-0.4/interface/gen-java/org/apache/cassandra/service/Cassandra.java:810145-834239,834349-834350 /incubator/cassandra/branches/cassandra-0.5/interface/gen-java/org/apache/cassandra/service/Cassandra.java:72-903502 Propchange: cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java -- --- svn:mergeinfo (original) +++ svn:mergeinfo Mon Dec 27 22:19:01 2010 @@ -1,5 +1,5 @@ /cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:922689-1051640,1051662 -/cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:1026517-1052542 +/cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:1026517-1053205 /incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/column_t.java:774578-792198 /incubator/cassandra/branches/cassandra-0.4/interface/gen-java/org/apache/cassandra/service/Column.java:810145-834239,834349-834350 /incubator/cassandra/branches/cassandra-0.5/interface/gen-java/org/apache/cassandra/service/Column.java:72-903502 Propchange: cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java -- --- svn:mergeinfo (original) +++ svn:mergeinfo Mon Dec 27 22:19:01 2010 @@ -1,5 +1,5 @@ /cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java:922689-1051640,1051662 -/cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java:1026517-1052542 +/cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java:1026517-1053205
[jira] Commented: (CASSANDRA-1871) Expose index-building status in JMX + cli schema description
[ https://issues.apache.org/jira/browse/CASSANDRA-1871?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975358#action_12975358 ] Jonathan Ellis commented on CASSANDRA-1871: --- is this patch against trunk? can you rebase to 0.7? Expose index-building status in JMX + cli schema description Key: CASSANDRA-1871 URL: https://issues.apache.org/jira/browse/CASSANDRA-1871 Project: Cassandra Issue Type: Improvement Components: Tools Affects Versions: 0.7 beta 1 Reporter: Jonathan Ellis Assignee: Pavel Yaskevich Priority: Minor Fix For: 0.7.1 Attachments: CASSANDRA-1871.patch This involves 1) exposing a JMX method to fetch a list of indexes that are built (should check SystemTable.isIndexBuilt -- see CFS.addIndex for example use) 2) incorporating that into the cli display 3) add index name to IndexBuilder.getTaskType. cli can then fetch active task from CompactionManager; if it is an index build, show % complete as CompactionInfo (i.e. IndexBuilder) BytesRead/TotalBytes. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1903) NullPointerException from o.a.c.db.ReplicateOnWriteTask
[ https://issues.apache.org/jira/browse/CASSANDRA-1903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975359#action_12975359 ] Eric Evans commented on CASSANDRA-1903: --- Upon further examination... The null value here is row.cf, it's null because the read came up empty, so this looks like a race to me (the read is beating the write). But... the bigger question is, why have we added a read to the write path? This seems very wrong to me. Also, what does ReplicateOnWriteTask do that the existing replication doesn't? How come we're doing both? This all seems to have changed as part of the counters implementation, so it would be great if someone involved with that could sound off on this. NullPointerException from o.a.c.db.ReplicateOnWriteTask --- Key: CASSANDRA-1903 URL: https://issues.apache.org/jira/browse/CASSANDRA-1903 Project: Cassandra Issue Type: Bug Components: Core Reporter: Eric Evans Attachments: thrift-test.py I'm seeing a whole lot of these when writing to a node. {noformat} java.lang.NullPointerException at org.apache.cassandra.db.ReplicateOnWriteTask.run(ReplicateOnWriteTask.java:97) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) at java.lang.Thread.run(Thread.java:636) {noformat} I don't think it will be difficult to reproduce, but the script I'm using is attached. I bisected the source tree and and http://svn.apache.org/viewvc?view=revrevision=1052356 seems to the culprit (a merge from 0.7). Maybe CASSANDRA-1530? -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (CASSANDRA-1453) stress.java
[ https://issues.apache.org/jira/browse/CASSANDRA-1453?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-1453: -- Reviewer: brandon.williams (was: jbellis) How does this look now, Brandon? Do you still see stress.java using more bandwidth than equivalent stress.py? (This would probably indicate a problem w/ stress.java metrics.) stress.java --- Key: CASSANDRA-1453 URL: https://issues.apache.org/jira/browse/CASSANDRA-1453 Project: Cassandra Issue Type: New Feature Components: Tools Reporter: Jonathan Ellis Assignee: Pavel Yaskevich Priority: Minor Fix For: 0.7.1 Attachments: stress-v2.patch, stress-v3.patch, stress.patch stress.py seems to scale poorly past a fairly small number of threads/processes. (against a 3-node, RF=1 cluster, I got 3x as much throughput with 4 machines running stress.py -t 32, as running 1 with -t 128. these were 8-core client machines, and -t 128 reported only 50% cpu used.) since we ship with the thrift java api pre-built, this would also mean not making people build thrift before using the stress test, which is at best a pain for newcomers and on Windows a major hurdle. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1896) Improve throughput by adding buffering to the inter-node TCP communication
[ https://issues.apache.org/jira/browse/CASSANDRA-1896?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975362#action_12975362 ] Tsiki commented on CASSANDRA-1896: -- Jonathan. The code was based on 0.6.5. This used limit(). I agree with you that it is buggy. And if I am not mistaken it was corrected in 0.6.8. And so the patch needs to be adjusted as well. As I have not implemented my changed into 0.6.8 or 0.7, I would agree that it is best to try it first and compare it to the results with a BB. Brandon. I ended up with a 6GB heap and 256MB for the young gen (though I think the latest test I did was with 128MB young gen). As for your test, running virtualized may also contribute to low utilization. Perhaps it is worth a check on real HW alone. Improve throughput by adding buffering to the inter-node TCP communication -- Key: CASSANDRA-1896 URL: https://issues.apache.org/jira/browse/CASSANDRA-1896 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 0.6 Reporter: Tsiki Assignee: Brandon Williams Fix For: 0.6.9, 0.7.1 Attachments: 1896.txt The inbound and outbound TCP implementation under org.apache.cassandra.net does not buffer the socket streams. A simple change in IncomingTcpConnection and OutboundTcpConnection may give a rather big throughput increase. In my tests, I got up t o 30% more out of my cluster. Below is the diff of these two files with buffering included. The diff is over release 0.6.5 but can be quite simply applied also to 0.7. I suggest perhaps to limit the buffered input stream I added in IncomingTcpConnection to 4K. The Outbound implementation can surely be implemented a bit better (remove some of the code I duplicated there). diff -r apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/IncomingTcpConnection.java fix/apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/IncomingTcpConnection.java 44c44 input = new DataInputStream(new BufferedInputStream(socket.getInputStream())); --- input = new DataInputStream(socket.getInputStream()); diff -r apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/OutboundTcpConnection.java fix/apache-cassandra-0.6.5-src/src/java/org/apache/cassandra/net/OutboundTcpConnection.java 77d76 byte[] buf = new byte[4096]; 80,112c79,89 int l = 0; ByteBuffer bb; while ((bb = queue.peek()) != null l+bb.limit() buf.length) { bb = take(); System.arraycopy(bb.array(), 0, buf, l, bb.limit()); l += bb.limit(); } if (l == 0) { bb = take(); if (bb == CLOSE_SENTINEL) { disconnect(); continue; } if (socket != null || connect()) writeConnected(bb); else // clear out the queue, else gossip messages back up. queue.clear(); } else { if (socket != null || connect()) { try { output.write(buf, 0, l); if (queue.peek() == null) output.flush(); } catch (IOException e) { logger.info(error writing to + endpoint); disconnect(); } } else { queue.clear(); } } --- ByteBuffer bb = take(); if (bb == CLOSE_SENTINEL) { disconnect(); continue; } if (socket != null || connect()) writeConnected(bb); else // clear out the queue, else gossip messages back up. queue.clear(); -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Assigned: (CASSANDRA-982) read repair on quorum consistencylevel
[ https://issues.apache.org/jira/browse/CASSANDRA-982?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis reassigned CASSANDRA-982: Assignee: Jonathan Ellis Now that RR is done on the coordinator node (CASSANDRA-1873) it should be fairly easy to combine quorum read / read repair: create a ReadResponseResolver for the read, and send the data to the client when desired CL is achieved; if CL is less than CL.ALL, also schedule a RR check in TIMEOUT ms, that will resolve + repair responses that come after the response is ready. (Alternatively, trigger the check in ResponseVerbHandler for better latency.) If RR is disabled, then only send the request to the minimum nodes needed to satisfy CL as in Daniel's patch. I will give this a try after CASSANDRA-1905 is done, which touches a lot of the same code. read repair on quorum consistencylevel -- Key: CASSANDRA-982 URL: https://issues.apache.org/jira/browse/CASSANDRA-982 Project: Cassandra Issue Type: New Feature Components: Core Reporter: Jonathan Ellis Assignee: Jonathan Ellis Priority: Minor Fix For: 0.7.1 Attachments: 0001-better-digest-checking-for-ReadResponseResolver.patch, 0002-quorum-only-read.txt, 982-resolve-digests-v2.txt CASSANDRA-930 made read repair fuzzy optional, but this only helps with ConsistencyLevel.ONE: - Quorum reads always send requests to all nodes - only the first Quorum's worth of responses get compared So what we'd like to do two changes: - only send read requests to the closest R live nodes - if read repair is enabled, also compare results from the other nodes in the background -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (CASSANDRA-982) read repair on quorum consistencylevel
[ https://issues.apache.org/jira/browse/CASSANDRA-982?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-982: - Remaining Estimate: 6h Original Estimate: 6h read repair on quorum consistencylevel -- Key: CASSANDRA-982 URL: https://issues.apache.org/jira/browse/CASSANDRA-982 Project: Cassandra Issue Type: New Feature Components: Core Reporter: Jonathan Ellis Assignee: Jonathan Ellis Priority: Minor Fix For: 0.7.1 Attachments: 0001-better-digest-checking-for-ReadResponseResolver.patch, 0002-quorum-only-read.txt, 982-resolve-digests-v2.txt Original Estimate: 6h Remaining Estimate: 6h CASSANDRA-930 made read repair fuzzy optional, but this only helps with ConsistencyLevel.ONE: - Quorum reads always send requests to all nodes - only the first Quorum's worth of responses get compared So what we'd like to do two changes: - only send read requests to the closest R live nodes - if read repair is enabled, also compare results from the other nodes in the background -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1907) AbstractCassandraDaemon blows up when log4j config is specified using a physical file.
[ https://issues.apache.org/jira/browse/CASSANDRA-1907?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975365#action_12975365 ] Hudson commented on CASSANDRA-1907: --- Integrated in Cassandra-0.7 #120 (See [https://hudson.apache.org/hudson/job/Cassandra-0.7/120/]) log4j configuration wasn't handling configurations specified by URL. patch by Gary Dusbabek, reviewed by Jake Luciani. CASSANDRA-1907 AbstractCassandraDaemon blows up when log4j config is specified using a physical file. -- Key: CASSANDRA-1907 URL: https://issues.apache.org/jira/browse/CASSANDRA-1907 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.7.0 rc 2 Reporter: Gary Dusbabek Assignee: Gary Dusbabek Priority: Minor Fix For: 0.7.1 Attachments: 0001-1907.patch -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Resolved: (CASSANDRA-1306) Forcing a major compaction on an CF with indexed sub columns occasionally causes an error
[ https://issues.apache.org/jira/browse/CASSANDRA-1306?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis resolved CASSANDRA-1306. --- Resolution: Cannot Reproduce Assignee: (was: Jonathan Ellis) I think this was fixed post-beta1 Forcing a major compaction on an CF with indexed sub columns occasionally causes an error - Key: CASSANDRA-1306 URL: https://issues.apache.org/jira/browse/CASSANDRA-1306 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.7 beta 1 Reporter: Nate McCall Note that is is a function of ColumnFamilyStore's forceMajorCompaction ERROR 14:33:31,430 Uncaught exception in thread Thread[CompactionExecutor:1,5,main] java.util.concurrent.ExecutionException: java.io.IOException: Keys must be written in ascending order. at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222) at java.util.concurrent.FutureTask.get(FutureTask.java:83) at org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.afterExecute(DebuggableThreadPoolExecutor.java:87) at org.apache.cassandra.db.CompactionManager$CompactionExecutor.afterExecute(CompactionManager.java:659) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:888) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:619) Caused by: java.io.IOException: Keys must be written in ascending order. at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:69) at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:86) at org.apache.cassandra.db.CompactionManager.doCompaction(CompactionManager.java:350) at org.apache.cassandra.db.CompactionManager$5.call(CompactionManager.java:227) 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) ... 2 more -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (CASSANDRA-1788) reduce copies on read, write paths
[ https://issues.apache.org/jira/browse/CASSANDRA-1788?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-1788: -- Remaining Estimate: 24h Original Estimate: 24h reduce copies on read, write paths -- Key: CASSANDRA-1788 URL: https://issues.apache.org/jira/browse/CASSANDRA-1788 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Jonathan Ellis Assignee: Jonathan Ellis Priority: Minor Fix For: 0.7.1 Attachments: 0001-setup.txt, 0002-remove-copies-from-network-path.txt, 1788-v2.txt, 1788-v3.txt, 1788-v4.txt, 1788.txt Original Estimate: 24h Remaining Estimate: 24h Currently, we do _three_ unnecessary copies (that is, writing to the socket is necessary; any other copies made are overhead) for each message: - constructing the Message body byte[] (this is typically a call to a ICompactSerializer[2] serialize method, but sometimes we cheat e.g. in SchemaCheckVerbHandler's reply) - which is copied to a buffer containing the entire Message (i.e. including Header) when sendOneWay calls Message.serializer.serialize() - which is copied to a newly-allocated ByteBuffer when sendOneWay calls packIt - which is what we write to the socket For deserialize we perform a similar orgy of copies: - IncomingTcpConnection reads the Message length, allocates a byte[], and reads the serialized Message into it - ITcpC then calls Message.serializer().deserialize, which allocates a new byte[] for the body and copies that part - finally, the verbHandler (determined by the now-deserialized Message header) deserializes the actual object represented by the body Most of these are out of scope for 0.7 but I think we can at least elide the last copy on the write path and the first on the read. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
svn commit: r1053213 - in /cassandra/branches/cassandra-0.6: CHANGES.txt src/java/org/apache/cassandra/net/OutboundTcpConnection.java
Author: jbellis Date: Mon Dec 27 22:45:32 2010 New Revision: 1053213 URL: http://svn.apache.org/viewvc?rev=1053213view=rev Log: enable keepalive on intra-cluster sockets patch by jbellis; reviewed by brandonwilliams for CASSANDRA-1766 Modified: cassandra/branches/cassandra-0.6/CHANGES.txt cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/OutboundTcpConnection.java Modified: cassandra/branches/cassandra-0.6/CHANGES.txt URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/CHANGES.txt?rev=1053213r1=1053212r2=1053213view=diff == --- cassandra/branches/cassandra-0.6/CHANGES.txt (original) +++ cassandra/branches/cassandra-0.6/CHANGES.txt Mon Dec 27 22:45:32 2010 @@ -28,6 +28,7 @@ * Re-cache hot keys post-compaction without hitting disk (CASSANDRA-1878) * manage read repair in coordinator instead of data source, to provide latency information to dynamic snitch (CASSANDRA-1873) + * enable keepalive on intra-cluster sockets (CASSANDRA-1766) 0.6.8 Modified: cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/OutboundTcpConnection.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/OutboundTcpConnection.java?rev=1053213r1=1053212r2=1053213view=diff == --- cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/OutboundTcpConnection.java (original) +++ cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/OutboundTcpConnection.java Mon Dec 27 22:45:32 2010 @@ -148,6 +148,7 @@ public class OutboundTcpConnection exten { // zero means 'bind on any available port.' socket = new Socket(endpoint, DatabaseDescriptor.getStoragePort(), FBUtilities.getLocalAddress(), 0); +socket.setKeepAlive(true); socket.setTcpNoDelay(true); output = new DataOutputStream(socket.getOutputStream()); return true;
[jira] Updated: (CASSANDRA-1903) NullPointerException from o.a.c.db.ReplicateOnWriteTask
[ https://issues.apache.org/jira/browse/CASSANDRA-1903?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eric Evans updated CASSANDRA-1903: -- Assignee: Kelvin Kakugawa NullPointerException from o.a.c.db.ReplicateOnWriteTask --- Key: CASSANDRA-1903 URL: https://issues.apache.org/jira/browse/CASSANDRA-1903 Project: Cassandra Issue Type: Bug Components: Core Reporter: Eric Evans Assignee: Kelvin Kakugawa Attachments: thrift-test.py I'm seeing a whole lot of these when writing to a node. {noformat} java.lang.NullPointerException at org.apache.cassandra.db.ReplicateOnWriteTask.run(ReplicateOnWriteTask.java:97) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) at java.lang.Thread.run(Thread.java:636) {noformat} I don't think it will be difficult to reproduce, but the script I'm using is attached. I bisected the source tree and and http://svn.apache.org/viewvc?view=revrevision=1052356 seems to the culprit (a merge from 0.7). Maybe CASSANDRA-1530? -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1766) Streaming never makes progress
[ https://issues.apache.org/jira/browse/CASSANDRA-1766?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975372#action_12975372 ] Hudson commented on CASSANDRA-1766: --- Integrated in Cassandra-0.6 #36 (See [https://hudson.apache.org/hudson/job/Cassandra-0.6/36/]) enable keepalive on intra-cluster sockets patch by jbellis; reviewed by brandonwilliams for CASSANDRA-1766 Streaming never makes progress -- Key: CASSANDRA-1766 URL: https://issues.apache.org/jira/browse/CASSANDRA-1766 Project: Cassandra Issue Type: Bug Affects Versions: 0.6 Reporter: Brandon Williams Assignee: Jonathan Ellis Fix For: 0.6.9, 0.7.1 Attachments: 1766-keepalive.txt, CASSANDRA-1766.patch I have a client that can never complete a bootstrap. AC finishes, streaming begins. Stream initiate completes, and the sources wait on the transfer to finish, but progress is never made on any stream. Nodetool reports streaming is happening, the socket is held open, but nothing happens. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
svn commit: r1053219 - in /cassandra/branches/cassandra-0.7: ./ interface/thrift/gen-java/org/apache/cassandra/thrift/ src/java/org/apache/cassandra/net/
Author: jbellis Date: Mon Dec 27 23:10:39 2010 New Revision: 1053219 URL: http://svn.apache.org/viewvc?rev=1053219view=rev Log: merge from 0.6 Modified: cassandra/branches/cassandra-0.7/ (props changed) cassandra/branches/cassandra-0.7/CHANGES.txt cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java (props changed) cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java (props changed) cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java (props changed) cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/NotFoundException.java (props changed) cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/SuperColumn.java (props changed) cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/net/MessagingService.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/net/OutboundTcpConnection.java Propchange: cassandra/branches/cassandra-0.7/ -- --- svn:mergeinfo (original) +++ svn:mergeinfo Mon Dec 27 23:10:39 2010 @@ -1,4 +1,4 @@ -/cassandra/branches/cassandra-0.6:922689-1051640,1051662,1052357 +/cassandra/branches/cassandra-0.6:922689-1053218 /cassandra/branches/cassandra-0.7:1035666,1050269 /cassandra/trunk:1026516-1026734,1028929 /incubator/cassandra/branches/cassandra-0.3:774578-796573 Modified: cassandra/branches/cassandra-0.7/CHANGES.txt URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/CHANGES.txt?rev=1053219r1=1053218r2=1053219view=diff == --- cassandra/branches/cassandra-0.7/CHANGES.txt (original) +++ cassandra/branches/cassandra-0.7/CHANGES.txt Mon Dec 27 23:10:39 2010 @@ -10,6 +10,7 @@ dev and seq scan operations (CASSANDRA-1470) * add OpenBitSet to support larger bloom filters (CASSANDRA-1555) * handle URL-specified log4j regression (CASSANDRA-1907) + * enable keepalive on intra-cluster sockets (CASSANDRA-1766) 0.7.0-rc3 Propchange: cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java -- --- svn:mergeinfo (original) +++ svn:mergeinfo Mon Dec 27 23:10:39 2010 @@ -1,4 +1,4 @@ -/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:922689-1051640,1051662,1052357 +/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:922689-1053218 /cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:1035666,1050269 /cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:1026516-1026734,1028929 /incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/Cassandra.java:774578-796573 Propchange: cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java -- --- svn:mergeinfo (original) +++ svn:mergeinfo Mon Dec 27 23:10:39 2010 @@ -1,4 +1,4 @@ -/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:922689-1051640,1051662,1052357 +/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:922689-1053218 /cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:1035666,1050269 /cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:1026516-1026734,1028929 /incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/column_t.java:774578-792198 Propchange: cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java -- --- svn:mergeinfo (original) +++ svn:mergeinfo Mon Dec 27 23:10:39 2010 @@ -1,4 +1,4 @@ -/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java:922689-1051640,1051662,1052357 +/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java:922689-1053218 /cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java:1035666,1050269 /cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java:1026516-1026734,1028929 /incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/InvalidRequestException.java:774578-796573
svn commit: r1053220 - /cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
Author: jbellis Date: Mon Dec 27 23:17:05 2010 New Revision: 1053220 URL: http://svn.apache.org/viewvc?rev=1053220view=rev Log: make constant fields final patch by jbellis Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java?rev=1053220r1=1053219r2=1053220view=diff == --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java Mon Dec 27 23:17:05 2010 @@ -39,17 +39,17 @@ import org.apache.cassandra.utils.FBUtil */ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean { -private static int UPDATES_PER_INTERVAL = 1; -private static int UPDATE_INTERVAL_IN_MS = DatabaseDescriptor.getDynamicUpdateInterval(); -private static int RESET_INTERVAL_IN_MS = DatabaseDescriptor.getDynamicResetInterval(); -private static double BADNESS_THRESHOLD = DatabaseDescriptor.getDynamicBadnessThreshold(); -private static int WINDOW_SIZE = 100; +private static final int UPDATES_PER_INTERVAL = 1; +private static final int UPDATE_INTERVAL_IN_MS = DatabaseDescriptor.getDynamicUpdateInterval(); +private static final int RESET_INTERVAL_IN_MS = DatabaseDescriptor.getDynamicResetInterval(); +private static final double BADNESS_THRESHOLD = DatabaseDescriptor.getDynamicBadnessThreshold(); +private static final int WINDOW_SIZE = 100; private boolean registered = false; -private ConcurrentHashMapInetAddress, Double scores = new ConcurrentHashMap(); -private ConcurrentHashMapInetAddress, AdaptiveLatencyTracker windows = new ConcurrentHashMap(); -private AtomicInteger intervalupdates = new AtomicInteger(0); -public IEndpointSnitch subsnitch; +private final ConcurrentHashMapInetAddress, Double scores = new ConcurrentHashMapInetAddress, Double(); +private final ConcurrentHashMapInetAddress, AdaptiveLatencyTracker windows = new ConcurrentHashMapInetAddress, AdaptiveLatencyTracker(); +private final AtomicInteger intervalupdates = new AtomicInteger(0); +public final IEndpointSnitch subsnitch; public DynamicEndpointSnitch(IEndpointSnitch snitch) { @@ -213,8 +213,8 @@ public class DynamicEndpointSnitch exten /** a threadsafe version of BoundedStatsDeque+ArrivalWindow with modification for arbitrary times **/ class AdaptiveLatencyTracker extends AbstractStatsDeque { -private LinkedBlockingDequeDouble latencies; -private static double SENTINEL_COMPARE = 0.0001; // arbitrary; as long as it is the same across hosts it doesn't matter +private final LinkedBlockingDequeDouble latencies; +private static final double SENTINEL_COMPARE = 0.0001; // arbitrary; as long as it is the same across hosts it doesn't matter AdaptiveLatencyTracker(int size) {
[jira] Updated: (CASSANDRA-1903) NullPointerException from o.a.c.db.ReplicateOnWriteTask
[ https://issues.apache.org/jira/browse/CASSANDRA-1903?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gary Dusbabek updated CASSANDRA-1903: - Attachment: 1903.diff NullPointerException from o.a.c.db.ReplicateOnWriteTask --- Key: CASSANDRA-1903 URL: https://issues.apache.org/jira/browse/CASSANDRA-1903 Project: Cassandra Issue Type: Bug Components: Core Reporter: Eric Evans Assignee: Kelvin Kakugawa Attachments: 1903.diff, thrift-test.py I'm seeing a whole lot of these when writing to a node. {noformat} java.lang.NullPointerException at org.apache.cassandra.db.ReplicateOnWriteTask.run(ReplicateOnWriteTask.java:97) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) at java.lang.Thread.run(Thread.java:636) {noformat} I don't think it will be difficult to reproduce, but the script I'm using is attached. I bisected the source tree and and http://svn.apache.org/viewvc?view=revrevision=1052356 seems to the culprit (a merge from 0.7). Maybe CASSANDRA-1530? -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Resolved: (CASSANDRA-1903) NullPointerException from o.a.c.db.ReplicateOnWriteTask
[ https://issues.apache.org/jira/browse/CASSANDRA-1903?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gary Dusbabek resolved CASSANDRA-1903. -- Resolution: Fixed Fix Version/s: 0.8 committed. NullPointerException from o.a.c.db.ReplicateOnWriteTask --- Key: CASSANDRA-1903 URL: https://issues.apache.org/jira/browse/CASSANDRA-1903 Project: Cassandra Issue Type: Bug Components: Core Reporter: Eric Evans Assignee: Kelvin Kakugawa Fix For: 0.8 Attachments: 1903.diff, thrift-test.py I'm seeing a whole lot of these when writing to a node. {noformat} java.lang.NullPointerException at org.apache.cassandra.db.ReplicateOnWriteTask.run(ReplicateOnWriteTask.java:97) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) at java.lang.Thread.run(Thread.java:636) {noformat} I don't think it will be difficult to reproduce, but the script I'm using is attached. I bisected the source tree and and http://svn.apache.org/viewvc?view=revrevision=1052356 seems to the culprit (a merge from 0.7). Maybe CASSANDRA-1530? -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1905) count timeouts towards dynamicsnitch latencies
[ https://issues.apache.org/jira/browse/CASSANDRA-1905?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975381#action_12975381 ] Brandon Williams commented on CASSANDRA-1905: - +1 count timeouts towards dynamicsnitch latencies -- Key: CASSANDRA-1905 URL: https://issues.apache.org/jira/browse/CASSANDRA-1905 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.6.6 Reporter: Jonathan Ellis Assignee: Jonathan Ellis Fix For: 0.6.9, 0.7.1 Attachments: 1905.txt receiveTiming is only called by ResponseVerbHandler; we need to add timing information for timed-out requests as well. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Resolved: (CASSANDRA-1909) normal replication shouldn't happen on counter CFs.
[ https://issues.apache.org/jira/browse/CASSANDRA-1909?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kelvin Kakugawa resolved CASSANDRA-1909. Resolution: Not A Problem normal replication shouldn't happen on counter CFs. --- Key: CASSANDRA-1909 URL: https://issues.apache.org/jira/browse/CASSANDRA-1909 Project: Cassandra Issue Type: Bug Components: Core Reporter: Gary Dusbabek Assignee: Kelvin Kakugawa Fix For: 0.8 -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1909) normal replication shouldn't happen on counter CFs.
[ https://issues.apache.org/jira/browse/CASSANDRA-1909?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975386#action_12975386 ] Kelvin Kakugawa commented on CASSANDRA-1909: read repair is allowed for counter CFs, as well. However, using both replicate-on-write and read repair may be excessive. I'd recommend defaulting to read repair, unless you require stronger consistency for a read penalty on write. normal replication shouldn't happen on counter CFs. --- Key: CASSANDRA-1909 URL: https://issues.apache.org/jira/browse/CASSANDRA-1909 Project: Cassandra Issue Type: Bug Components: Core Reporter: Gary Dusbabek Assignee: Kelvin Kakugawa Fix For: 0.8 -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1903) NullPointerException from o.a.c.db.ReplicateOnWriteTask
[ https://issues.apache.org/jira/browse/CASSANDRA-1903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975388#action_12975388 ] Eric Evans commented on CASSANDRA-1903: --- So the NullPointerExceptions are normal? NullPointerException from o.a.c.db.ReplicateOnWriteTask --- Key: CASSANDRA-1903 URL: https://issues.apache.org/jira/browse/CASSANDRA-1903 Project: Cassandra Issue Type: Bug Components: Core Reporter: Eric Evans Assignee: Kelvin Kakugawa Fix For: 0.8 Attachments: 1903.diff, thrift-test.py I'm seeing a whole lot of these when writing to a node. {noformat} java.lang.NullPointerException at org.apache.cassandra.db.ReplicateOnWriteTask.run(ReplicateOnWriteTask.java:97) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) at java.lang.Thread.run(Thread.java:636) {noformat} I don't think it will be difficult to reproduce, but the script I'm using is attached. I bisected the source tree and and http://svn.apache.org/viewvc?view=revrevision=1052356 seems to the culprit (a merge from 0.7). Maybe CASSANDRA-1530? -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (CASSANDRA-1871) Expose index-building status in JMX + cli schema description
[ https://issues.apache.org/jira/browse/CASSANDRA-1871?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Pavel Yaskevich updated CASSANDRA-1871: --- Attachment: CASSANDRA-1871-0.7.patch Work branch 0.7, latest commit (5b9abe18549f97f77d7f0b82af7c68546e3f696c - make constant fields final) Expose index-building status in JMX + cli schema description Key: CASSANDRA-1871 URL: https://issues.apache.org/jira/browse/CASSANDRA-1871 Project: Cassandra Issue Type: Improvement Components: Tools Affects Versions: 0.7 beta 1 Reporter: Jonathan Ellis Assignee: Pavel Yaskevich Priority: Minor Fix For: 0.7.1 Attachments: CASSANDRA-1871-0.7.patch, CASSANDRA-1871.patch This involves 1) exposing a JMX method to fetch a list of indexes that are built (should check SystemTable.isIndexBuilt -- see CFS.addIndex for example use) 2) incorporating that into the cli display 3) add index name to IndexBuilder.getTaskType. cli can then fetch active task from CompactionManager; if it is an index build, show % complete as CompactionInfo (i.e. IndexBuilder) BytesRead/TotalBytes. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1903) NullPointerException from o.a.c.db.ReplicateOnWriteTask
[ https://issues.apache.org/jira/browse/CASSANDRA-1903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975400#action_12975400 ] Hudson commented on CASSANDRA-1903: --- Integrated in Cassandra #642 (See [https://hudson.apache.org/hudson/job/Cassandra/642/]) set DEFAULT_REPLICATE_ON_WRITE to false. patch by gdusbabek, reviewed by eevans. CASSANDRA-1903 NullPointerException from o.a.c.db.ReplicateOnWriteTask --- Key: CASSANDRA-1903 URL: https://issues.apache.org/jira/browse/CASSANDRA-1903 Project: Cassandra Issue Type: Bug Components: Core Reporter: Eric Evans Assignee: Kelvin Kakugawa Fix For: 0.8 Attachments: 1903.diff, thrift-test.py I'm seeing a whole lot of these when writing to a node. {noformat} java.lang.NullPointerException at org.apache.cassandra.db.ReplicateOnWriteTask.run(ReplicateOnWriteTask.java:97) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) at java.lang.Thread.run(Thread.java:636) {noformat} I don't think it will be difficult to reproduce, but the script I'm using is attached. I bisected the source tree and and http://svn.apache.org/viewvc?view=revrevision=1052356 seems to the culprit (a merge from 0.7). Maybe CASSANDRA-1530? -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
svn commit: r1053244 - in /cassandra/branches/cassandra-0.6: ./ src/java/org/apache/cassandra/net/ src/java/org/apache/cassandra/utils/
Author: jbellis Date: Tue Dec 28 03:10:26 2010 New Revision: 1053244 URL: http://svn.apache.org/viewvc?rev=1053244view=rev Log: count timeouts towards dynamicsnitch latencies patch by jbellis; reviewed by brandonwilliams for CASSANDRA-1905 Added: cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/IMessageCallback.java Modified: cassandra/branches/cassandra-0.6/CHANGES.txt cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/AsyncResult.java cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/IAsyncCallback.java cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/IAsyncResult.java cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/MessagingService.java cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/ResponseVerbHandler.java cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/utils/ExpiringMap.java Modified: cassandra/branches/cassandra-0.6/CHANGES.txt URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/CHANGES.txt?rev=1053244r1=1053243r2=1053244view=diff == --- cassandra/branches/cassandra-0.6/CHANGES.txt (original) +++ cassandra/branches/cassandra-0.6/CHANGES.txt Tue Dec 28 03:10:26 2010 @@ -29,6 +29,7 @@ * manage read repair in coordinator instead of data source, to provide latency information to dynamic snitch (CASSANDRA-1873) * enable keepalive on intra-cluster sockets (CASSANDRA-1766) + * count timeouts towards dynamicsnitch latencies (CASSANDRA-1905) 0.6.8 Modified: cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/AsyncResult.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/AsyncResult.java?rev=1053244r1=1053243r2=1053244view=diff == --- cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/AsyncResult.java (original) +++ cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/AsyncResult.java Tue Dec 28 03:10:26 2010 @@ -121,6 +121,8 @@ class AsyncResult implements IAsyncResul { lock_.unlock(); } + +MessagingService.removeRegisteredCallback(response.getMessageId()); } public InetAddress getFrom() Modified: cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/IAsyncCallback.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/IAsyncCallback.java?rev=1053244r1=1053243r2=1053244view=diff == --- cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/IAsyncCallback.java (original) +++ cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/IAsyncCallback.java Tue Dec 28 03:10:26 2010 @@ -24,7 +24,7 @@ package org.apache.cassandra.net; * service. In particular, if any shared state is referenced, making * response alone synchronized will not suffice. */ -public interface IAsyncCallback +public interface IAsyncCallback extends IMessageCallback { /** * @param msg response received. Modified: cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/IAsyncResult.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/IAsyncResult.java?rev=1053244r1=1053243r2=1053244view=diff == --- cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/IAsyncResult.java (original) +++ cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/IAsyncResult.java Tue Dec 28 03:10:26 2010 @@ -22,7 +22,7 @@ import java.net.InetAddress; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -public interface IAsyncResult +public interface IAsyncResult extends IMessageCallback { /** * This is used to check if the task has been completed Added: cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/IMessageCallback.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/IMessageCallback.java?rev=1053244view=auto == --- cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/IMessageCallback.java (added) +++ cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/IMessageCallback.java Tue Dec 28 03:10:26 2010 @@ -0,0 +1,10 @@ +package org.apache.cassandra.net; + +public interface IMessageCallback +{ +} +package org.apache.cassandra.net; + +public interface IMessageCallback +{ +} Modified: cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/MessagingService.java URL:
buildbot failure in ASF Buildbot on cassandra-0.6
The Buildbot has detected a new failure of cassandra-0.6 on ASF Buildbot. Full details are available at: http://ci.apache.org/builders/cassandra-0.6/builds/263 Buildbot URL: http://ci.apache.org/ Buildslave for this Build: isis_ubuntu Build Reason: Build Source Stamp: [branch cassandra/branches/cassandra-0.6] 1053244 Blamelist: jbellis BUILD FAILED: failed compile sincerely, -The Buildbot
svn commit: r1053245 - in /cassandra/branches/cassandra-0.7: ./ interface/thrift/gen-java/org/apache/cassandra/thrift/ src/java/org/apache/cassandra/net/ src/java/org/apache/cassandra/utils/
Author: jbellis Date: Tue Dec 28 03:17:24 2010 New Revision: 1053245 URL: http://svn.apache.org/viewvc?rev=1053245view=rev Log: merged from 0.6 Added: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/net/IMessageCallback.java - copied, changed from r1053244, cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/IMessageCallback.java Modified: cassandra/branches/cassandra-0.7/ (props changed) cassandra/branches/cassandra-0.7/CHANGES.txt cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java (props changed) cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java (props changed) cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java (props changed) cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/NotFoundException.java (props changed) cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/SuperColumn.java (props changed) cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/net/AsyncResult.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/net/IAsyncCallback.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/net/IAsyncResult.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/net/MessagingService.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/net/ResponseVerbHandler.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/ExpiringMap.java Propchange: cassandra/branches/cassandra-0.7/ -- --- svn:mergeinfo (original) +++ svn:mergeinfo Tue Dec 28 03:17:24 2010 @@ -1,4 +1,4 @@ -/cassandra/branches/cassandra-0.6:922689-1053218 +/cassandra/branches/cassandra-0.6:922689-1053244 /cassandra/branches/cassandra-0.7:1035666,1050269 /cassandra/trunk:1026516-1026734,1028929 /incubator/cassandra/branches/cassandra-0.3:774578-796573 Modified: cassandra/branches/cassandra-0.7/CHANGES.txt URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/CHANGES.txt?rev=1053245r1=1053244r2=1053245view=diff == --- cassandra/branches/cassandra-0.7/CHANGES.txt (original) +++ cassandra/branches/cassandra-0.7/CHANGES.txt Tue Dec 28 03:17:24 2010 @@ -11,6 +11,7 @@ dev * add OpenBitSet to support larger bloom filters (CASSANDRA-1555) * handle URL-specified log4j regression (CASSANDRA-1907) * enable keepalive on intra-cluster sockets (CASSANDRA-1766) + * count timeouts towards dynamicsnitch latencies (CASSANDRA-1905) 0.7.0-rc3 Propchange: cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java -- --- svn:mergeinfo (original) +++ svn:mergeinfo Tue Dec 28 03:17:24 2010 @@ -1,4 +1,4 @@ -/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:922689-1053218 +/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:922689-1053244 /cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:1035666,1050269 /cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:1026516-1026734,1028929 /incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/Cassandra.java:774578-796573 Propchange: cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java -- --- svn:mergeinfo (original) +++ svn:mergeinfo Tue Dec 28 03:17:24 2010 @@ -1,4 +1,4 @@ -/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:922689-1053218 +/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:922689-1053244 /cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:1035666,1050269 /cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:1026516-1026734,1028929 /incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/column_t.java:774578-792198 Propchange: cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java -- --- svn:mergeinfo (original) +++ svn:mergeinfo Tue Dec 28 03:17:24 2010 @@ -1,4 +1,4 @@
svn commit: r1053246 - in /cassandra/trunk: ./ contrib/stress/ contrib/stress/bin/ contrib/stress/src/ contrib/stress/src/org/ contrib/stress/src/org/apache/ contrib/stress/src/org/apache/cassandra/ c
Author: jbellis Date: Tue Dec 28 03:19:34 2010 New Revision: 1053246 URL: http://svn.apache.org/viewvc?rev=1053246view=rev Log: merge from 0.7 Added: cassandra/trunk/contrib/stress/ - copied from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/ cassandra/trunk/contrib/stress/bin/ - copied from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/bin/ cassandra/trunk/contrib/stress/bin/stress - copied unchanged from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/bin/stress cassandra/trunk/contrib/stress/build.xml - copied unchanged from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/build.xml cassandra/trunk/contrib/stress/src/ - copied from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/src/ cassandra/trunk/contrib/stress/src/org/ - copied from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/src/org/ cassandra/trunk/contrib/stress/src/org/apache/ - copied from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/src/org/apache/ cassandra/trunk/contrib/stress/src/org/apache/cassandra/ - copied from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/src/org/apache/cassandra/ cassandra/trunk/contrib/stress/src/org/apache/cassandra/contrib/ - copied from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/src/org/apache/cassandra/contrib/ cassandra/trunk/contrib/stress/src/org/apache/cassandra/contrib/stress/ - copied from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/src/org/apache/cassandra/contrib/stress/ cassandra/trunk/contrib/stress/src/org/apache/cassandra/contrib/stress/Session.java - copied unchanged from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/src/org/apache/cassandra/contrib/stress/Session.java cassandra/trunk/contrib/stress/src/org/apache/cassandra/contrib/stress/Stress.java - copied unchanged from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/src/org/apache/cassandra/contrib/stress/Stress.java cassandra/trunk/contrib/stress/src/org/apache/cassandra/contrib/stress/tests/ - copied from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/src/org/apache/cassandra/contrib/stress/tests/ cassandra/trunk/contrib/stress/src/org/apache/cassandra/contrib/stress/tests/IndexedRangeSlicer.java - copied unchanged from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/src/org/apache/cassandra/contrib/stress/tests/IndexedRangeSlicer.java cassandra/trunk/contrib/stress/src/org/apache/cassandra/contrib/stress/tests/Inserter.java - copied unchanged from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/src/org/apache/cassandra/contrib/stress/tests/Inserter.java cassandra/trunk/contrib/stress/src/org/apache/cassandra/contrib/stress/tests/MultiGetter.java - copied unchanged from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/src/org/apache/cassandra/contrib/stress/tests/MultiGetter.java cassandra/trunk/contrib/stress/src/org/apache/cassandra/contrib/stress/tests/RangeSlicer.java - copied unchanged from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/src/org/apache/cassandra/contrib/stress/tests/RangeSlicer.java cassandra/trunk/contrib/stress/src/org/apache/cassandra/contrib/stress/tests/Reader.java - copied unchanged from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/src/org/apache/cassandra/contrib/stress/tests/Reader.java cassandra/trunk/contrib/stress/src/org/apache/cassandra/contrib/stress/util/ - copied from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/src/org/apache/cassandra/contrib/stress/util/ cassandra/trunk/contrib/stress/src/org/apache/cassandra/contrib/stress/util/OperationThread.java - copied unchanged from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/src/org/apache/cassandra/contrib/stress/util/OperationThread.java cassandra/trunk/contrib/stress/src/org/apache/cassandra/contrib/stress/util/Range.java - copied unchanged from r1053221, cassandra/branches/cassandra-0.7/contrib/stress/src/org/apache/cassandra/contrib/stress/util/Range.java cassandra/trunk/src/java/org/apache/cassandra/net/IMessageCallback.java - copied unchanged from r1053245, cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/net/IMessageCallback.java Modified: cassandra/trunk/ (props changed) cassandra/trunk/CHANGES.txt 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)
svn commit: r1053247 - in /cassandra/branches/cassandra-0.7: ./ src/java/org/apache/cassandra/cli/ src/java/org/apache/cassandra/db/
Author: jbellis Date: Tue Dec 28 03:24:41 2010 New Revision: 1053247 URL: http://svn.apache.org/viewvc?rev=1053247view=rev Log: Expose index-building status in JMX + cli schema description patch by Pavel Yaskevich; reviewed by jbellis for CASSANDRA-1871 Modified: cassandra/branches/cassandra-0.7/CHANGES.txt cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/cli/CliClient.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/cli/CliOptions.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/cli/CliSessionState.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/Table.java Modified: cassandra/branches/cassandra-0.7/CHANGES.txt URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/CHANGES.txt?rev=1053247r1=1053246r2=1053247view=diff == --- cassandra/branches/cassandra-0.7/CHANGES.txt (original) +++ cassandra/branches/cassandra-0.7/CHANGES.txt Tue Dec 28 03:24:41 2010 @@ -12,6 +12,8 @@ dev * handle URL-specified log4j regression (CASSANDRA-1907) * enable keepalive on intra-cluster sockets (CASSANDRA-1766) * count timeouts towards dynamicsnitch latencies (CASSANDRA-1905) + * Expose index-building status in JMX + cli schema description + (CASSANDRA-1871) 0.7.0-rc3 Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/cli/CliClient.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/cli/CliClient.java?rev=1053247r1=1053246r2=1053247view=diff == --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/cli/CliClient.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/cli/CliClient.java Tue Dec 28 03:24:41 2010 @@ -17,6 +17,7 @@ */ package org.apache.cassandra.cli; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.*; @@ -25,8 +26,11 @@ import com.google.common.base.Charsets; import org.antlr.runtime.tree.Tree; import org.apache.cassandra.auth.SimpleAuthenticator; import org.apache.cassandra.config.ConfigurationException; +import org.apache.cassandra.db.ColumnFamilyStoreMBean; +import org.apache.cassandra.db.CompactionManagerMBean; import org.apache.cassandra.db.marshal.*; import org.apache.cassandra.thrift.*; +import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.UUIDGen; @@ -1206,6 +1210,11 @@ public class CliClient extends CliUserHe private void describeKeySpace(String keySpaceName, KsDef metadata) throws TException { +NodeProbe probe = sessionState.getNodeProbe(); + +// getting compaction manager MBean to displaying index building information +CompactionManagerMBean compactionManagerMBean = (probe == null) ? null : probe.getCompactionManagerProxy(); + // Describe and display sessionState.out.println(Keyspace: + keySpaceName + :); try @@ -1213,10 +1222,12 @@ public class CliClient extends CliUserHe KsDef ks_def; ks_def = metadata == null ? thriftClient.describe_keyspace(keySpaceName) : metadata; sessionState.out.println( Replication Strategy: + ks_def.strategy_class); + if (ks_def.strategy_class.endsWith(.NetworkTopologyStrategy)) sessionState.out.println(Options: + FBUtilities.toString(ks_def.strategy_options)); else sessionState.out.println(Replication Factor: + ks_def.replication_factor); + sessionState.out.println( Column Families:); boolean isSuper; @@ -1224,6 +1235,9 @@ public class CliClient extends CliUserHe Collections.sort(ks_def.cf_defs, new CfDefNamesComparator()); for (CfDef cf_def : ks_def.cf_defs) { +// fetching bean for current column family store +ColumnFamilyStoreMBean cfMBean = (probe == null) ? null : probe.getCfsProxy(ks_def.getName(), cf_def.getName()); + isSuper = cf_def.column_type.equals(Super); sessionState.out.printf(ColumnFamily: %s%s%n, cf_def.name, isSuper ? (Super) : ); @@ -1241,6 +1255,12 @@ public class CliClient extends CliUserHe sessionState.out.printf( Compaction min/max thresholds: %s/%s%n, cf_def.min_compaction_threshold, cf_def.max_compaction_threshold); sessionState.out.printf( Read repair chance: %s%n, cf_def.read_repair_chance); +// if we have connection to the
svn commit: r1053248 - in /cassandra/trunk: ./ interface/thrift/gen-java/org/apache/cassandra/thrift/ src/java/org/apache/cassandra/cli/ src/java/org/apache/cassandra/db/
Author: jbellis Date: Tue Dec 28 03:31:56 2010 New Revision: 1053248 URL: http://svn.apache.org/viewvc?rev=1053248view=rev Log: merge from 0.7 Modified: cassandra/trunk/ (props changed) cassandra/trunk/CHANGES.txt 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/CliOptions.java cassandra/trunk/src/java/org/apache/cassandra/cli/CliSessionState.java cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java cassandra/trunk/src/java/org/apache/cassandra/db/Table.java Propchange: cassandra/trunk/ -- --- svn:mergeinfo (original) +++ svn:mergeinfo Tue Dec 28 03:31:56 2010 @@ -1,5 +1,5 @@ /cassandra/branches/cassandra-0.6:922689-1052356,1052358-1053244 -/cassandra/branches/cassandra-0.7:1026517-1053245 +/cassandra/branches/cassandra-0.7:1026517-1053247 /incubator/cassandra/branches/cassandra-0.3:774578-796573 /incubator/cassandra/branches/cassandra-0.4:810145-834239,834349-834350 /incubator/cassandra/branches/cassandra-0.5:72-915439 Modified: cassandra/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/cassandra/trunk/CHANGES.txt?rev=1053248r1=1053247r2=1053248view=diff == --- cassandra/trunk/CHANGES.txt (original) +++ cassandra/trunk/CHANGES.txt Tue Dec 28 03:31:56 2010 @@ -17,6 +17,8 @@ * handle URL-specified log4j regression (CASSANDRA-1907) * enable keepalive on intra-cluster sockets (CASSANDRA-1766) * count timeouts towards dynamicsnitch latencies (CASSANDRA-1905) + * Expose index-building status in JMX + cli schema description + (CASSANDRA-1871) 0.7.0-rc3 Propchange: cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java -- --- svn:mergeinfo (original) +++ svn:mergeinfo Tue Dec 28 03:31:56 2010 @@ -1,5 +1,5 @@ /cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:922689-1052356,1052358-1053244 -/cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:1026517-1053245 +/cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:1026517-1053247 /incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/Cassandra.java:774578-796573 /incubator/cassandra/branches/cassandra-0.4/interface/gen-java/org/apache/cassandra/service/Cassandra.java:810145-834239,834349-834350 /incubator/cassandra/branches/cassandra-0.5/interface/gen-java/org/apache/cassandra/service/Cassandra.java:72-903502 Propchange: cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java -- --- svn:mergeinfo (original) +++ svn:mergeinfo Tue Dec 28 03:31:56 2010 @@ -1,5 +1,5 @@ /cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:922689-1052356,1052358-1053244 -/cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:1026517-1053245 +/cassandra/branches/cassandra-0.7/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:1026517-1053247 /incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/column_t.java:774578-792198 /incubator/cassandra/branches/cassandra-0.4/interface/gen-java/org/apache/cassandra/service/Column.java:810145-834239,834349-834350 /incubator/cassandra/branches/cassandra-0.5/interface/gen-java/org/apache/cassandra/service/Column.java:72-903502 Propchange: cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java -- --- svn:mergeinfo (original) +++ svn:mergeinfo Tue Dec 28 03:31:56 2010 @@ -1,5 +1,5 @@ /cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java:922689-1052356,1052358-1053244
[jira] Commented: (CASSANDRA-1891) large supercolumn deserialization invokes CSLM worst case scenario
[ https://issues.apache.org/jira/browse/CASSANDRA-1891?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975410#action_12975410 ] Jonathan Ellis commented on CASSANDRA-1891: --- bq. I've never been able to get cassandra's unit tests to work locally As in they don't run, or as in they fail? Pretty much all the core devs (and Hudson) run these constantly, I'm not aware of any long-standing problems there. large supercolumn deserialization invokes CSLM worst case scenario -- Key: CASSANDRA-1891 URL: https://issues.apache.org/jira/browse/CASSANDRA-1891 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Cliff Moon Priority: Minor Fix For: 0.7.1 Attachments: supercolumn.patch SuperColumn deserialization hits a worst case insert scenario for CSLM: inserting pre-sorted entries one at a time. Inside of CSLM this requires scanning to the end of the list and doing a comparison at every step for every item inserted. This patch supplies a SortedMap interface to the supercolumn deserialization. CSLM will do a bulk insert from a SortedMap interface supplied in the constructor. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (CASSANDRA-1871) Expose index-building status in JMX + cli schema description
[ https://issues.apache.org/jira/browse/CASSANDRA-1871?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12975415#action_12975415 ] Hudson commented on CASSANDRA-1871: --- Integrated in Cassandra-0.7 #123 (See [https://hudson.apache.org/hudson/job/Cassandra-0.7/123/]) Expose index-building status in JMX + cli schema description patch by Pavel Yaskevich; reviewed by jbellis for CASSANDRA-1871 Expose index-building status in JMX + cli schema description Key: CASSANDRA-1871 URL: https://issues.apache.org/jira/browse/CASSANDRA-1871 Project: Cassandra Issue Type: Improvement Components: Tools Affects Versions: 0.7 beta 1 Reporter: Jonathan Ellis Assignee: Pavel Yaskevich Priority: Minor Fix For: 0.7.1 Attachments: CASSANDRA-1871-0.7.patch, CASSANDRA-1871.patch This involves 1) exposing a JMX method to fetch a list of indexes that are built (should check SystemTable.isIndexBuilt -- see CFS.addIndex for example use) 2) incorporating that into the cli display 3) add index name to IndexBuilder.getTaskType. cli can then fetch active task from CompactionManager; if it is an index build, show % complete as CompactionInfo (i.e. IndexBuilder) BytesRead/TotalBytes. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (CASSANDRA-1901) getRestrictedRanges bug where node owns minimum token
[ https://issues.apache.org/jira/browse/CASSANDRA-1901?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stu Hood updated CASSANDRA-1901: Attachment: 0001-Switch-minimum-token-for-RP-to-1-for-midpoint-purposes.txt Switches the minimum token for RP to -1. Fixes the case mentioned on the mailing list, but hasn't been subjected to the same scrutiny as the OPPs get in StorageProxyTest: the singletons make this excessively difficult. getRestrictedRanges bug where node owns minimum token - Key: CASSANDRA-1901 URL: https://issues.apache.org/jira/browse/CASSANDRA-1901 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.6.9, 0.7.0 rc 2 Reporter: Jonathan Ellis Assignee: Stu Hood Fix For: 0.6.9, 0.7.1 Attachments: 0001-Switch-minimum-token-for-RP-to-1-for-midpoint-purposes.txt From the ML, there are two RF=1 nodes, 0 for the local node (17.224.36.17) and 85070591730234615865843651857942052864 for the remote node (17.224.109.80). Debug log shows {code} DEBUG [pool-1-thread-4] 2010-12-23 12:54:26,958 CassandraServer.java (line 479) range_slice DEBUG [pool-1-thread-4] 2010-12-23 12:54:26,958 StorageProxy.java (line 412) RangeSliceCommand{keyspace='Harvest', column_family='TestCentroids', super_column=null, predicate=SlicePredicate(slice_range:SliceRange(start:80 01 00 01 00 00 00 10 67 65 74 5F 72 61 6E 67 65 5F 73 6C 69 63 65 73 00 00 00 0C 0C 00 01 0B 00 03 00 00 00 0D 54 65 73 74 43 65 6E 74 72 6F 69 64 73 00 0C 00 02 0C 00 02 0B 00 01 00 00 00 00, finish:80 01 00 01 00 00 00 10 67 65 74 5F 72 61 6E 67 65 5F 73 6C 69 63 65 73 00 00 00 0C 0C 00 01 0B 00 03 00 00 00 0D 54 65 73 74 43 65 6E 74 72 6F 69 64 73 00 0C 00 02 0C 00 02 0B 00 01 00 00 00 00 0B 00 02 00 00 00 00, reversed:false, count:1)), range=[0,0], max_keys=11} DEBUG [pool-1-thread-4] 2010-12-23 12:54:26,958 StorageProxy.java (line 597) restricted ranges for query [0,0] are [[0,0]] DEBUG [pool-1-thread-4] 2010-12-23 12:54:26,959 StorageProxy.java (line 423) === endpoint: belize1.apple.com/17.224.36.17 for range.right 0 {code} Thus, node 85070591730234615865843651857942052864 is left out. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Assigned: (CASSANDRA-1895) Loadbalance in trunk leaves cluster in bad state
[ https://issues.apache.org/jira/browse/CASSANDRA-1895?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stu Hood reassigned CASSANDRA-1895: --- Assignee: Stu Hood (was: Nick Bailey) I think the burden of proof is back on me now, so I'll try and reproduce this once I'm back in the office tomorrow. Loadbalance in trunk leaves cluster in bad state Key: CASSANDRA-1895 URL: https://issues.apache.org/jira/browse/CASSANDRA-1895 Project: Cassandra Issue Type: Bug Affects Versions: 0.8 Reporter: Stu Hood Assignee: Stu Hood Priority: Critical Fix For: 0.8 Attachments: logs.tgz, ring-views.txt Running loadbalance against a node in a 4 node cluster leaves gossip in a wonky state. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.