Monitoring move progress

2011-12-21 Thread Ethan Rowe
I've got some nodes in a moving state in a cluster (the nodes to which
they stream shouldn't overlap), and I'm finding it difficult to determine
if they're actually doing anything related to the move at this point, or if
they're stuck in the state and not actually doing anything.

In each case, I issued the move command per usual.

The log shows information about the move when it begins, showing the
correct token change that I would expect in each case.

Compactions took place on each moving node, which can be viewed through
nodetool compactionstats or through the CompactionManager in JMX.

But eventually the compactions stopped, apart from various ongoing
secondary index rebuilds and consequent related index compactions.  Yet I
see no stream transfers via netstats.  My expectation is that after the
compactions (which the project wiki refers to as anti-compactions), I
would start to see outbound streaming activity in netstats.  Yet I do not.

I don't see any errors listed in the logs on the moving servers since the
moves began.

Using cassandra 1.0.5.  ByteOrderedPartitioner.

Any suggestions on how to determine what's going on?

Thanks in advance.
- Ethan


New node unable to stream (0.8.5)

2011-09-15 Thread Ethan Rowe
Hi.

We've been running a 7-node cluster with RF 3, QUORUM reads/writes in our
production environment for a few months.  It's been consistently stable
during this period, particularly once we got out maintenance strategy fully
worked out (per node, one repair a week, one major compaction a week, the
latter due to the nature of our data model and usage).  While this cluster
started, back in June or so, on the 0.7 series, it's been running 0.8.3 for
a while now with no issues.  We upgraded to 0.8.5 two days ago, having
tested the upgrade in our staging cluster (with an otherwise identical
configuration) previously and verified that our application's various use
cases appeared successful.

One of our nodes suffered a disk failure yesterday.  We attempted to replace
the dead node by placing a new node at OldNode.initial_token - 1 with
auto_bootstrap on.  A few things went awry from there:

1. We never saw the new node in bootstrap mode; it became available pretty
much immediately upon joining the ring, and never reported a joining
state.  I did verify that auto_bootstrap was on.

2. I mistakenly ran repair on the new node rather than removetoken on the
old node, due to a delightful mental error.  The repair got nowhere fast, as
it attempts to repair against the down node which throws an exception.  So I
interrupted the repair, restarted the node to clear any pending validation
compactions, and...

3. Ran removetoken for the old node.

4. We let this run for some time and saw eventually that all the nodes
appeared to be done various compactions and were stuck at streaming.  Many
streams listed as open, none making any progress.

5.  I observed an Rpc-related exception on the new node (where the
removetoken was launched) and concluded that the streams were broken so the
process wouldn't ever finish.

6. Ran a removetoken force to get the dead node out of the mix.  No
problems.

7. Ran a repair on the new node.

8. Validations ran, streams opened up, and again things got stuck in
streaming, hanging for over an hour with no progress.

9. Musing that lingering tasks from the removetoken could be a factor, I
performed a rolling restart and attempted a repair again.

10. Same problem.  Did another rolling restart and attempted a fresh repair
on the most important column family alone.

11. Same problem.  Streams included CFs not specified, so I guess they must
be for hinted handoff.

In concluding that streaming is stuck, I've observed:
- streams will be open to the new node from other nodes, but the new node
doesn't list them
- streams will be open to the other nodes from the new node, but the other
nodes don't list them
- the streams reported may make some initial progress, but then they hang at
a particular point and do not move on for an hour or more.
- The logs report repair-related activity, until NPEs on incoming TCP
connections show up, which appear likely to be the culprit.

I can provide more exact details when I'm done commuting.

With streaming broken on this node, I'm unable to run repairs, which is
obviously problematic.  The application didn't suffer any operational issues
as a consequence of this, but I need to review the overnight results to
verify we're not suffering data loss (I doubt we are).

At this point, I'm considering a couple options:
1. Remove the new node and let the adjacent node take over its range
2. Bring the new node down, add a new one in front of it, and properly
removetoken the problematic one.
3. Bring the new node down, remove all its data except for the system
keyspace, then bring it back up and repair it.
4. Revert to 0.8.3 and see if that helps.

Recommendations?

Thanks.
- Ethan


Re: New node unable to stream (0.8.5)

2011-09-15 Thread Ethan Rowe
Here's a typical log slice (not terribly informative, I fear):

  INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106 AntiEntropyService.java
 (l
 ine 884) Performing streaming repair of 1003 ranges with /10.34.90.8 for
 (299
 90798416657667504332586989223299634,54296681768153272037430773234349600451]
  INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java (line
 181)
 Stream context metadata
 [/mnt/cassandra/data/events_production/FitsByShip-g-1
 0-Data.db sections=88 progress=0/11707163 - 0%,
 /mnt/cassandra/data/events_pr
 oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 - 0%,
 /mnt/c
 assandra/data/events_production/FitsByShip-g-6-Data.db sections=1
 progress=0/
 6918814 - 0%, /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db
 s
 ections=260 progress=0/9091780 - 0%], 4 sstables.
  INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428 StreamOutSession.java
 (lin
 e 174) Streaming to /10.34.90.8
 ERROR [Thread-56] 2011-09-15 05:41:38,515 AbstractCassandraDaemon.java
 (line
 139) Fatal exception in thread Thread[Thread-56,5,main]
 java.lang.NullPointerException
 at
 org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC
 onnection.java:174)
 at
 org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn
 ection.java:114)



Not sure if the exception is related to the outbound streaming above; other
nodes are actively trying to stream to this node, so perhaps it comes from
those and temporal adjacency to the outbound stream is just coincidental.  I
have other snippets that look basically identical to the above, except if I
look at the logs to which this node is trying to stream, I see that it has
concurrently opened a stream in the other direction, which could be the one
that the exception pertains to.


On Thu, Sep 15, 2011 at 7:41 AM, Sylvain Lebresne sylv...@datastax.comwrote:

 On Thu, Sep 15, 2011 at 1:16 PM, Ethan Rowe et...@the-rowes.com wrote:
  Hi.
 
  We've been running a 7-node cluster with RF 3, QUORUM reads/writes in our
  production environment for a few months.  It's been consistently stable
  during this period, particularly once we got out maintenance strategy
 fully
  worked out (per node, one repair a week, one major compaction a week, the
  latter due to the nature of our data model and usage).  While this
 cluster
  started, back in June or so, on the 0.7 series, it's been running 0.8.3
 for
  a while now with no issues.  We upgraded to 0.8.5 two days ago, having
  tested the upgrade in our staging cluster (with an otherwise identical
  configuration) previously and verified that our application's various use
  cases appeared successful.
 
  One of our nodes suffered a disk failure yesterday.  We attempted to
 replace
  the dead node by placing a new node at OldNode.initial_token - 1 with
  auto_bootstrap on.  A few things went awry from there:
 
  1. We never saw the new node in bootstrap mode; it became available
 pretty
  much immediately upon joining the ring, and never reported a joining
  state.  I did verify that auto_bootstrap was on.
 
  2. I mistakenly ran repair on the new node rather than removetoken on the
  old node, due to a delightful mental error.  The repair got nowhere fast,
 as
  it attempts to repair against the down node which throws an exception.
  So I
  interrupted the repair, restarted the node to clear any pending
 validation
  compactions, and...
 
  3. Ran removetoken for the old node.
 
  4. We let this run for some time and saw eventually that all the nodes
  appeared to be done various compactions and were stuck at streaming.
 Many
  streams listed as open, none making any progress.
 
  5.  I observed an Rpc-related exception on the new node (where the
  removetoken was launched) and concluded that the streams were broken so
 the
  process wouldn't ever finish.
 
  6. Ran a removetoken force to get the dead node out of the mix.  No
  problems.
 
  7. Ran a repair on the new node.
 
  8. Validations ran, streams opened up, and again things got stuck in
  streaming, hanging for over an hour with no progress.
 
  9. Musing that lingering tasks from the removetoken could be a factor, I
  performed a rolling restart and attempted a repair again.
 
  10. Same problem.  Did another rolling restart and attempted a fresh
 repair
  on the most important column family alone.
 
  11. Same problem.  Streams included CFs not specified, so I guess they
 must
  be for hinted handoff.
 
  In concluding that streaming is stuck, I've observed:
  - streams will be open to the new node from other nodes, but the new node
  doesn't list them
  - streams will be open to the other nodes from the new node, but the
 other
  nodes don't list them
  - the streams reported may make some initial progress, but then they hang
 at
  a particular point and do not move on for an hour or more.
  - The logs report repair-related activity, until NPEs on incoming TCP
  connections show up, which appear likely

Re: New node unable to stream (0.8.5)

2011-09-15 Thread Ethan Rowe
I just noticed the following from one of Jonathan Ellis' messages yesterday:

 Added to NEWS:

- After upgrading, run nodetool scrub against each node before running
  repair, moving nodes, or adding new ones.


We did not do this, as it was not indicated as necessary in the news when we
were dealing with the upgrade.

So perhaps I need to scrub everything before going any further, though the
question is what to do with the problematic node.  Additionally, it would be
helpful to know if scrub will affect the hinted handoffs that have
accumulated, as these seem likely to be part of the set of failing streams.

On Thu, Sep 15, 2011 at 8:13 AM, Ethan Rowe et...@the-rowes.com wrote:

 Here's a typical log slice (not terribly informative, I fear):

  INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106 AntiEntropyService.java
 (l
 ine 884) Performing streaming repair of 1003 ranges with /10.34.90.8 for
 (299

 90798416657667504332586989223299634,54296681768153272037430773234349600451]
  INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java (line
 181)
 Stream context metadata
 [/mnt/cassandra/data/events_production/FitsByShip-g-1
 0-Data.db sections=88 progress=0/11707163 - 0%,
 /mnt/cassandra/data/events_pr
 oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 - 0%,
 /mnt/c
 assandra/data/events_production/FitsByShip-g-6-Data.db sections=1
 progress=0/
 6918814 - 0%,
 /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db s
 ections=260 progress=0/9091780 - 0%], 4 sstables.
  INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428 StreamOutSession.java
 (lin
 e 174) Streaming to /10.34.90.8
 ERROR [Thread-56] 2011-09-15 05:41:38,515 AbstractCassandraDaemon.java
 (line
 139) Fatal exception in thread Thread[Thread-56,5,main]
 java.lang.NullPointerException
 at
 org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC
 onnection.java:174)
 at
 org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn
 ection.java:114)



 Not sure if the exception is related to the outbound streaming above; other
 nodes are actively trying to stream to this node, so perhaps it comes from
 those and temporal adjacency to the outbound stream is just coincidental.  I
 have other snippets that look basically identical to the above, except if I
 look at the logs to which this node is trying to stream, I see that it has
 concurrently opened a stream in the other direction, which could be the one
 that the exception pertains to.


 On Thu, Sep 15, 2011 at 7:41 AM, Sylvain Lebresne sylv...@datastax.comwrote:

 On Thu, Sep 15, 2011 at 1:16 PM, Ethan Rowe et...@the-rowes.com wrote:
  Hi.
 
  We've been running a 7-node cluster with RF 3, QUORUM reads/writes in
 our
  production environment for a few months.  It's been consistently stable
  during this period, particularly once we got out maintenance strategy
 fully
  worked out (per node, one repair a week, one major compaction a week,
 the
  latter due to the nature of our data model and usage).  While this
 cluster
  started, back in June or so, on the 0.7 series, it's been running 0.8.3
 for
  a while now with no issues.  We upgraded to 0.8.5 two days ago, having
  tested the upgrade in our staging cluster (with an otherwise identical
  configuration) previously and verified that our application's various
 use
  cases appeared successful.
 
  One of our nodes suffered a disk failure yesterday.  We attempted to
 replace
  the dead node by placing a new node at OldNode.initial_token - 1 with
  auto_bootstrap on.  A few things went awry from there:
 
  1. We never saw the new node in bootstrap mode; it became available
 pretty
  much immediately upon joining the ring, and never reported a joining
  state.  I did verify that auto_bootstrap was on.
 
  2. I mistakenly ran repair on the new node rather than removetoken on
 the
  old node, due to a delightful mental error.  The repair got nowhere
 fast, as
  it attempts to repair against the down node which throws an exception.
  So I
  interrupted the repair, restarted the node to clear any pending
 validation
  compactions, and...
 
  3. Ran removetoken for the old node.
 
  4. We let this run for some time and saw eventually that all the nodes
  appeared to be done various compactions and were stuck at streaming.
 Many
  streams listed as open, none making any progress.
 
  5.  I observed an Rpc-related exception on the new node (where the
  removetoken was launched) and concluded that the streams were broken so
 the
  process wouldn't ever finish.
 
  6. Ran a removetoken force to get the dead node out of the mix.  No
  problems.
 
  7. Ran a repair on the new node.
 
  8. Validations ran, streams opened up, and again things got stuck in
  streaming, hanging for over an hour with no progress.
 
  9. Musing that lingering tasks from the removetoken could be a factor, I
  performed a rolling restart and attempted a repair again.
 
  10. Same problem.  Did another rolling

Re: New node unable to stream (0.8.5)

2011-09-15 Thread Ethan Rowe
After further review, I'm definitely going to scrub all the original nodes
in the cluster.

We've lost some data as a result of this situation.  It can be restored, but
the question is what to do with the problematic new node first.  I don't
particularly care about the data that's on it, since I'm going to re-import
the critical data from files anyway, and then I can recreate derivative data
afterwards.  So it's purely a matter of getting the cluster healthy again as
quickly as possible so I can begin that import process.

Any issue with running scrubs on multiple nodes at a time, provided they
aren't replication neighbors?

On Thu, Sep 15, 2011 at 8:18 AM, Ethan Rowe et...@the-rowes.com wrote:

 I just noticed the following from one of Jonathan Ellis' messages
 yesterday:

 Added to NEWS:

- After upgrading, run nodetool scrub against each node before running
  repair, moving nodes, or adding new ones.


 We did not do this, as it was not indicated as necessary in the news when
 we were dealing with the upgrade.

 So perhaps I need to scrub everything before going any further, though the
 question is what to do with the problematic node.  Additionally, it would be
 helpful to know if scrub will affect the hinted handoffs that have
 accumulated, as these seem likely to be part of the set of failing streams.

 On Thu, Sep 15, 2011 at 8:13 AM, Ethan Rowe et...@the-rowes.com wrote:

 Here's a typical log slice (not terribly informative, I fear):

  INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106
 AntiEntropyService.java (l
 ine 884) Performing streaming repair of 1003 ranges with /10.34.90.8 for
 (299

 90798416657667504332586989223299634,54296681768153272037430773234349600451]
  INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java (line
 181)
 Stream context metadata
 [/mnt/cassandra/data/events_production/FitsByShip-g-1
 0-Data.db sections=88 progress=0/11707163 - 0%,
 /mnt/cassandra/data/events_pr
 oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 - 0%,
 /mnt/c
 assandra/data/events_production/FitsByShip-g-6-Data.db sections=1
 progress=0/
 6918814 - 0%,
 /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db s
 ections=260 progress=0/9091780 - 0%], 4 sstables.
  INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428 StreamOutSession.java
 (lin
 e 174) Streaming to /10.34.90.8
 ERROR [Thread-56] 2011-09-15 05:41:38,515 AbstractCassandraDaemon.java
 (line
 139) Fatal exception in thread Thread[Thread-56,5,main]
 java.lang.NullPointerException
 at
 org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC
 onnection.java:174)
 at
 org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn
 ection.java:114)



 Not sure if the exception is related to the outbound streaming above;
 other nodes are actively trying to stream to this node, so perhaps it comes
 from those and temporal adjacency to the outbound stream is just
 coincidental.  I have other snippets that look basically identical to the
 above, except if I look at the logs to which this node is trying to stream,
 I see that it has concurrently opened a stream in the other direction, which
 could be the one that the exception pertains to.


 On Thu, Sep 15, 2011 at 7:41 AM, Sylvain Lebresne 
 sylv...@datastax.comwrote:

 On Thu, Sep 15, 2011 at 1:16 PM, Ethan Rowe et...@the-rowes.com wrote:
  Hi.
 
  We've been running a 7-node cluster with RF 3, QUORUM reads/writes in
 our
  production environment for a few months.  It's been consistently stable
  during this period, particularly once we got out maintenance strategy
 fully
  worked out (per node, one repair a week, one major compaction a week,
 the
  latter due to the nature of our data model and usage).  While this
 cluster
  started, back in June or so, on the 0.7 series, it's been running 0.8.3
 for
  a while now with no issues.  We upgraded to 0.8.5 two days ago, having
  tested the upgrade in our staging cluster (with an otherwise identical
  configuration) previously and verified that our application's various
 use
  cases appeared successful.
 
  One of our nodes suffered a disk failure yesterday.  We attempted to
 replace
  the dead node by placing a new node at OldNode.initial_token - 1 with
  auto_bootstrap on.  A few things went awry from there:
 
  1. We never saw the new node in bootstrap mode; it became available
 pretty
  much immediately upon joining the ring, and never reported a joining
  state.  I did verify that auto_bootstrap was on.
 
  2. I mistakenly ran repair on the new node rather than removetoken on
 the
  old node, due to a delightful mental error.  The repair got nowhere
 fast, as
  it attempts to repair against the down node which throws an exception.
  So I
  interrupted the repair, restarted the node to clear any pending
 validation
  compactions, and...
 
  3. Ran removetoken for the old node.
 
  4. We let this run for some time and saw eventually that all the nodes
  appeared to be done various

Re: New node unable to stream (0.8.5)

2011-09-15 Thread Ethan Rowe
Thanks, Jonathan.  I'll try the workaround and see if that gets the streams
flowing properly.

As I mentioned before, we did not run scrub yet.  What is the consequence of
letting the streams from the hinted handoffs complete if scrub hasn't been
run on these nodes?

I'm currently running scrub on one node to get a sense of the time frame.

Thanks again.
- Ethan

On Thu, Sep 15, 2011 at 9:09 AM, Jonathan Ellis jbel...@gmail.com wrote:

 That means we missed a place we needed to special-case for backwards
 compatibility -- the workaround is, add an empty encryption_options section
 to cassandra.yaml:

 encryption_options:
internode_encryption: none
keystore: conf/.keystore
keystore_password: cassandra
truststore: conf/.truststore
truststore_password: cassandra

 Created https://issues.apache.org/jira/browse/CASSANDRA-3212 to fix this.

 On Thu, Sep 15, 2011 at 7:13 AM, Ethan Rowe et...@the-rowes.com wrote:
  Here's a typical log slice (not terribly informative, I fear):
 
   INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106
 AntiEntropyService.java
  (l
  ine 884) Performing streaming repair of 1003 ranges with /10.34.90.8for
  (299
 
 
 90798416657667504332586989223299634,54296681768153272037430773234349600451]
   INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java (line
  181)
  Stream context metadata
  [/mnt/cassandra/data/events_production/FitsByShip-g-1
  0-Data.db sections=88 progress=0/11707163 - 0%,
  /mnt/cassandra/data/events_pr
  oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 - 0%,
  /mnt/c
  assandra/data/events_production/FitsByShip-g-6-Data.db sections=1
  progress=0/
  6918814 - 0%,
  /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db s
  ections=260 progress=0/9091780 - 0%], 4 sstables.
   INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428 StreamOutSession.java
  (lin
  e 174) Streaming to /10.34.90.8
  ERROR [Thread-56] 2011-09-15 05:41:38,515 AbstractCassandraDaemon.java
  (line
  139) Fatal exception in thread Thread[Thread-56,5,main]
  java.lang.NullPointerException
  at
  org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC
  onnection.java:174)
  at
  org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn
  ection.java:114)
 
  Not sure if the exception is related to the outbound streaming above;
 other
  nodes are actively trying to stream to this node, so perhaps it comes
 from
  those and temporal adjacency to the outbound stream is just coincidental.
  I
  have other snippets that look basically identical to the above, except if
 I
  look at the logs to which this node is trying to stream, I see that it
 has
  concurrently opened a stream in the other direction, which could be the
 one
  that the exception pertains to.
 
  On Thu, Sep 15, 2011 at 7:41 AM, Sylvain Lebresne sylv...@datastax.com
  wrote:
 
  On Thu, Sep 15, 2011 at 1:16 PM, Ethan Rowe et...@the-rowes.com
 wrote:
   Hi.
  
   We've been running a 7-node cluster with RF 3, QUORUM reads/writes in
   our
   production environment for a few months.  It's been consistently
 stable
   during this period, particularly once we got out maintenance strategy
   fully
   worked out (per node, one repair a week, one major compaction a week,
   the
   latter due to the nature of our data model and usage).  While this
   cluster
   started, back in June or so, on the 0.7 series, it's been running
 0.8.3
   for
   a while now with no issues.  We upgraded to 0.8.5 two days ago, having
   tested the upgrade in our staging cluster (with an otherwise identical
   configuration) previously and verified that our application's various
   use
   cases appeared successful.
  
   One of our nodes suffered a disk failure yesterday.  We attempted to
   replace
   the dead node by placing a new node at OldNode.initial_token - 1 with
   auto_bootstrap on.  A few things went awry from there:
  
   1. We never saw the new node in bootstrap mode; it became available
   pretty
   much immediately upon joining the ring, and never reported a joining
   state.  I did verify that auto_bootstrap was on.
  
   2. I mistakenly ran repair on the new node rather than removetoken on
   the
   old node, due to a delightful mental error.  The repair got nowhere
   fast, as
   it attempts to repair against the down node which throws an exception.
So I
   interrupted the repair, restarted the node to clear any pending
   validation
   compactions, and...
  
   3. Ran removetoken for the old node.
  
   4. We let this run for some time and saw eventually that all the nodes
   appeared to be done various compactions and were stuck at streaming.
   Many
   streams listed as open, none making any progress.
  
   5.  I observed an Rpc-related exception on the new node (where the
   removetoken was launched) and concluded that the streams were broken
 so
   the
   process wouldn't ever finish.
  
   6. Ran a removetoken force to get the dead node out of the mix

Re: New node unable to stream (0.8.5)

2011-09-15 Thread Ethan Rowe
On Thu, Sep 15, 2011 at 9:21 AM, Jonathan Ellis jbel...@gmail.com wrote:

 Where did the data loss come in?


The outcome of the analytical jobs run overnight while some of these repairs
were (not) running is consistent with what I would expect if perhaps 20-30%
of the source data was missing.  Given the strong consistency model we're
using, this is surprising to me, since the jobs did not report any read or
write failures.  I wonder if this is a consequence of the dead node missing
and the new node being operational but having received basically none of its
hinted handoff streams.  Perhaps with streaming fixed the data will
reappear, which would be a happy outcome, but if not, I can reimport the
critical stuff from files.

Scrub is safe to run in parallel.


Is it somewhat analogous to a major compaction in terms of I/O impact, with
perhaps less greedy use of disk space?


 On Thu, Sep 15, 2011 at 8:08 AM, Ethan Rowe et...@the-rowes.com wrote:
  After further review, I'm definitely going to scrub all the original
 nodes
  in the cluster.
  We've lost some data as a result of this situation.  It can be restored,
 but
  the question is what to do with the problematic new node first.  I don't
  particularly care about the data that's on it, since I'm going to
 re-import
  the critical data from files anyway, and then I can recreate derivative
 data
  afterwards.  So it's purely a matter of getting the cluster healthy again
 as
  quickly as possible so I can begin that import process.
  Any issue with running scrubs on multiple nodes at a time, provided they
  aren't replication neighbors?
  On Thu, Sep 15, 2011 at 8:18 AM, Ethan Rowe et...@the-rowes.com wrote:
 
  I just noticed the following from one of Jonathan Ellis' messages
  yesterday:
 
  Added to NEWS:
 
 - After upgrading, run nodetool scrub against each node before
 running
   repair, moving nodes, or adding new ones.
 
 
  We did not do this, as it was not indicated as necessary in the news
 when
  we were dealing with the upgrade.
  So perhaps I need to scrub everything before going any further, though
 the
  question is what to do with the problematic node.  Additionally, it
 would be
  helpful to know if scrub will affect the hinted handoffs that have
  accumulated, as these seem likely to be part of the set of failing
 streams.
  On Thu, Sep 15, 2011 at 8:13 AM, Ethan Rowe et...@the-rowes.com
 wrote:
 
  Here's a typical log slice (not terribly informative, I fear):
 
   INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106
  AntiEntropyService.java (l
  ine 884) Performing streaming repair of 1003 ranges with /10.34.90.8for
  (299
 
 
 90798416657667504332586989223299634,54296681768153272037430773234349600451]
   INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java
 (line
  181)
  Stream context metadata
  [/mnt/cassandra/data/events_production/FitsByShip-g-1
  0-Data.db sections=88 progress=0/11707163 - 0%,
  /mnt/cassandra/data/events_pr
  oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 - 0%,
  /mnt/c
  assandra/data/events_production/FitsByShip-g-6-Data.db sections=1
  progress=0/
  6918814 - 0%,
  /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db s
  ections=260 progress=0/9091780 - 0%], 4 sstables.
   INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428
 StreamOutSession.java
  (lin
  e 174) Streaming to /10.34.90.8
  ERROR [Thread-56] 2011-09-15 05:41:38,515 AbstractCassandraDaemon.java
  (line
  139) Fatal exception in thread Thread[Thread-56,5,main]
  java.lang.NullPointerException
  at
  org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC
  onnection.java:174)
  at
  org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn
  ection.java:114)
 
  Not sure if the exception is related to the outbound streaming above;
  other nodes are actively trying to stream to this node, so perhaps it
 comes
  from those and temporal adjacency to the outbound stream is just
  coincidental.  I have other snippets that look basically identical to
 the
  above, except if I look at the logs to which this node is trying to
 stream,
  I see that it has concurrently opened a stream in the other direction,
 which
  could be the one that the exception pertains to.
 
  On Thu, Sep 15, 2011 at 7:41 AM, Sylvain Lebresne 
 sylv...@datastax.com
  wrote:
 
  On Thu, Sep 15, 2011 at 1:16 PM, Ethan Rowe et...@the-rowes.com
 wrote:
   Hi.
  
   We've been running a 7-node cluster with RF 3, QUORUM reads/writes
 in
   our
   production environment for a few months.  It's been consistently
   stable
   during this period, particularly once we got out maintenance
 strategy
   fully
   worked out (per node, one repair a week, one major compaction a
 week,
   the
   latter due to the nature of our data model and usage).  While this
   cluster
   started, back in June or so, on the 0.7 series, it's been running
   0.8.3 for
   a while now with no issues.  We upgraded to 0.8.5 two days ago

Re: New node unable to stream (0.8.5)

2011-09-15 Thread Ethan Rowe
On Thu, Sep 15, 2011 at 10:03 AM, Jonathan Ellis jbel...@gmail.com wrote:

 If you added the new node as a seed, it would ignore bootstrap mode.
 And bootstrap / repair *do* use streaming so you'll want to re-run
 repair post-scrub.  (No need to re-bootstrap since you're repairing.)


Ah, of course.  That's what happened; the chef recipe added the node to its
own seed list, which is a problem I thought we'd fixed but apparently not.
 That definitely explains the bootstrap issue.  But no matter, so long as
the repairs can eventually run.


 Scrub is a little less heavyweight than major compaction but same
 ballpark.  It runs sstable-at-a-time so (as long as you haven't been
 in the habit of forcing majors) space should not be a concern.


Cool.  We've deactivated all tasks against these nodes and will scrub them
all in parallel, apply the encryption options you specified, and see where
that gets us.  Thanks for the assistance.
- Ethan


 On Thu, Sep 15, 2011 at 8:40 AM, Ethan Rowe et...@the-rowes.com wrote:
  On Thu, Sep 15, 2011 at 9:21 AM, Jonathan Ellis jbel...@gmail.com
 wrote:
 
  Where did the data loss come in?
 
  The outcome of the analytical jobs run overnight while some of these
 repairs
  were (not) running is consistent with what I would expect if perhaps
 20-30%
  of the source data was missing.  Given the strong consistency model we're
  using, this is surprising to me, since the jobs did not report any read
 or
  write failures.  I wonder if this is a consequence of the dead node
 missing
  and the new node being operational but having received basically none of
 its
  hinted handoff streams.  Perhaps with streaming fixed the data will
  reappear, which would be a happy outcome, but if not, I can reimport the
  critical stuff from files.
 
  Scrub is safe to run in parallel.
 
  Is it somewhat analogous to a major compaction in terms of I/O impact,
 with
  perhaps less greedy use of disk space?
 
 
  On Thu, Sep 15, 2011 at 8:08 AM, Ethan Rowe et...@the-rowes.com
 wrote:
   After further review, I'm definitely going to scrub all the original
   nodes
   in the cluster.
   We've lost some data as a result of this situation.  It can be
 restored,
   but
   the question is what to do with the problematic new node first.  I
 don't
   particularly care about the data that's on it, since I'm going to
   re-import
   the critical data from files anyway, and then I can recreate
 derivative
   data
   afterwards.  So it's purely a matter of getting the cluster healthy
   again as
   quickly as possible so I can begin that import process.
   Any issue with running scrubs on multiple nodes at a time, provided
 they
   aren't replication neighbors?
   On Thu, Sep 15, 2011 at 8:18 AM, Ethan Rowe et...@the-rowes.com
 wrote:
  
   I just noticed the following from one of Jonathan Ellis' messages
   yesterday:
  
   Added to NEWS:
  
  - After upgrading, run nodetool scrub against each node before
   running
repair, moving nodes, or adding new ones.
  
  
   We did not do this, as it was not indicated as necessary in the news
   when
   we were dealing with the upgrade.
   So perhaps I need to scrub everything before going any further,
 though
   the
   question is what to do with the problematic node.  Additionally, it
   would be
   helpful to know if scrub will affect the hinted handoffs that have
   accumulated, as these seem likely to be part of the set of failing
   streams.
   On Thu, Sep 15, 2011 at 8:13 AM, Ethan Rowe et...@the-rowes.com
   wrote:
  
   Here's a typical log slice (not terribly informative, I fear):
  
INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106
   AntiEntropyService.java (l
   ine 884) Performing streaming repair of 1003 ranges with /
 10.34.90.8
   for
   (299
  
  
  
 90798416657667504332586989223299634,54296681768153272037430773234349600451]
INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java
   (line
   181)
   Stream context metadata
   [/mnt/cassandra/data/events_production/FitsByShip-g-1
   0-Data.db sections=88 progress=0/11707163 - 0%,
   /mnt/cassandra/data/events_pr
   oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 -
   0%,
   /mnt/c
   assandra/data/events_production/FitsByShip-g-6-Data.db sections=1
   progress=0/
   6918814 - 0%,
   /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db s
   ections=260 progress=0/9091780 - 0%], 4 sstables.
INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428
   StreamOutSession.java
   (lin
   e 174) Streaming to /10.34.90.8
   ERROR [Thread-56] 2011-09-15 05:41:38,515
   AbstractCassandraDaemon.java
   (line
   139) Fatal exception in thread Thread[Thread-56,5,main]
   java.lang.NullPointerException
   at
   org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC
   onnection.java:174)
   at
   org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn
   ection.java:114)
  
   Not sure if the exception is related to the outbound

Re: New node unable to stream (0.8.5)

2011-09-15 Thread Ethan Rowe


 Cool.  We've deactivated all tasks against these nodes and will scrub them
 all in parallel, apply the encryption options you specified, and see where
 that gets us.  Thanks for the assistance.


To follow up:
* We scrubbed all the nodes
* We applied the encryption options specified
* A repair is continuing (for about an hour so far, perhaps more) on the
new, problematic node; it's successfully streaming data from its neighbors
and has built up a roughly equivalent data volume on disk

We'll see if the data is fully restored once this process completes.  Even
if it isn't, it seems likely that the cluster will be in a healthy state
soon, so we can reimport as necessary and we'll be out of the woods.

Now that I've said all that, something will inevitably go wrong, but until
that happens, thanks again for the feedback.
- Ethan



 On Thu, Sep 15, 2011 at 8:40 AM, Ethan Rowe et...@the-rowes.com wrote:
  On Thu, Sep 15, 2011 at 9:21 AM, Jonathan Ellis jbel...@gmail.com
 wrote:
 
  Where did the data loss come in?
 
  The outcome of the analytical jobs run overnight while some of these
 repairs
  were (not) running is consistent with what I would expect if perhaps
 20-30%
  of the source data was missing.  Given the strong consistency model
 we're
  using, this is surprising to me, since the jobs did not report any read
 or
  write failures.  I wonder if this is a consequence of the dead node
 missing
  and the new node being operational but having received basically none of
 its
  hinted handoff streams.  Perhaps with streaming fixed the data will
  reappear, which would be a happy outcome, but if not, I can reimport the
  critical stuff from files.
 
  Scrub is safe to run in parallel.
 
  Is it somewhat analogous to a major compaction in terms of I/O impact,
 with
  perhaps less greedy use of disk space?
 
 
  On Thu, Sep 15, 2011 at 8:08 AM, Ethan Rowe et...@the-rowes.com
 wrote:
   After further review, I'm definitely going to scrub all the original
   nodes
   in the cluster.
   We've lost some data as a result of this situation.  It can be
 restored,
   but
   the question is what to do with the problematic new node first.  I
 don't
   particularly care about the data that's on it, since I'm going to
   re-import
   the critical data from files anyway, and then I can recreate
 derivative
   data
   afterwards.  So it's purely a matter of getting the cluster healthy
   again as
   quickly as possible so I can begin that import process.
   Any issue with running scrubs on multiple nodes at a time, provided
 they
   aren't replication neighbors?
   On Thu, Sep 15, 2011 at 8:18 AM, Ethan Rowe et...@the-rowes.com
 wrote:
  
   I just noticed the following from one of Jonathan Ellis' messages
   yesterday:
  
   Added to NEWS:
  
  - After upgrading, run nodetool scrub against each node before
   running
repair, moving nodes, or adding new ones.
  
  
   We did not do this, as it was not indicated as necessary in the news
   when
   we were dealing with the upgrade.
   So perhaps I need to scrub everything before going any further,
 though
   the
   question is what to do with the problematic node.  Additionally, it
   would be
   helpful to know if scrub will affect the hinted handoffs that have
   accumulated, as these seem likely to be part of the set of failing
   streams.
   On Thu, Sep 15, 2011 at 8:13 AM, Ethan Rowe et...@the-rowes.com
   wrote:
  
   Here's a typical log slice (not terribly informative, I fear):
  
INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106
   AntiEntropyService.java (l
   ine 884) Performing streaming repair of 1003 ranges with /
 10.34.90.8
   for
   (299
  
  
  
 90798416657667504332586989223299634,54296681768153272037430773234349600451]
INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java
   (line
   181)
   Stream context metadata
   [/mnt/cassandra/data/events_production/FitsByShip-g-1
   0-Data.db sections=88 progress=0/11707163 - 0%,
   /mnt/cassandra/data/events_pr
   oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 -
   0%,
   /mnt/c
   assandra/data/events_production/FitsByShip-g-6-Data.db sections=1
   progress=0/
   6918814 - 0%,
   /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db s
   ections=260 progress=0/9091780 - 0%], 4 sstables.
INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428
   StreamOutSession.java
   (lin
   e 174) Streaming to /10.34.90.8
   ERROR [Thread-56] 2011-09-15 05:41:38,515
   AbstractCassandraDaemon.java
   (line
   139) Fatal exception in thread Thread[Thread-56,5,main]
   java.lang.NullPointerException
   at
   org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC
   onnection.java:174)
   at
   org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn
   ection.java:114)
  
   Not sure if the exception is related to the outbound streaming
 above;
   other nodes are actively trying to stream to this node, so perhaps
 it
   comes
   from

Truncate introspection

2011-06-27 Thread Ethan Rowe
Is there any straightforward means of seeing what's going on after issuing a
truncate (on 0.7.5)?  I'm not seeing evidence that anything actually
happened.  I've disabled read repair on the column family in question and
don't have anything actively reading/writing at present, apart from my
one-off tests to see if rows have disappeared.

Thanks in advance.


Re: Truncate introspection

2011-06-27 Thread Ethan Rowe
If those went to zero, it would certainly tell me something happened.  :)  I
guess watching that would be a way of seeing something was going on.

Is the truncate itself propagating a ring-wide marker or anything so the CF
is logically empty before being physically removed?  That's the impression
I got from the docs but it wasn't totally clear to me.

On Mon, Jun 27, 2011 at 3:33 PM, Jonathan Ellis jbel...@gmail.com wrote:

 There's a JMX method to get the number of sstables in a CF, is that
 what you're looking for?

 On Mon, Jun 27, 2011 at 1:04 PM, Ethan Rowe et...@the-rowes.com wrote:
  Is there any straightforward means of seeing what's going on after
 issuing a
  truncate (on 0.7.5)?  I'm not seeing evidence that anything actually
  happened.  I've disabled read repair on the column family in question and
  don't have anything actively reading/writing at present, apart from my
  one-off tests to see if rows have disappeared.
  Thanks in advance.



 --
 Jonathan Ellis
 Project Chair, Apache Cassandra
 co-founder of DataStax, the source for professional Cassandra support
 http://www.datastax.com



Re: What's the best modeling approach for ordering events by date?

2011-04-15 Thread Ethan Rowe
Hi.

So, the OPP will direct all activity for a range of keys to a particular
node (or set of nodes, in accordance with your replication factor).
 Depending on the volume of writes, this could be fine.  Depending on the
distribution of key values you write at any given time, it can also be fine.
 But if you're using the OPP, and your keys align with the time of receiving
the data, and your application writes that data as it receives it, you're
going to be placing write activity on effectively one node at a time, for
the range of time allocated to that node.

If you use RP, and can divide time into finer slices such that you have
multiple tweets in a row, you trade off a more complex read in exchange for
better distribution of load throughout your cluster.  The necessity of this
depends on your particulars.

In your TweetsBySecond example, you're using a deterministic set of keys
(the keys correspond to seconds since epoch).  Querying for ranges of time
is nice with OPP, but if the ranges of time you're interested in are
constrained, you don't specifically need OPP.  You could use RP and request
all the keys for the seconds contained within the time range of interest.
 In this way, you balance writes across the cluster more effectively than
you would with OPP, while still getting a workable data set.  Again, the
degree to which you need this is dependent on your situation.  Others on the
list will no doubt have more informed opinions on this than me.  :)

On Thu, Apr 14, 2011 at 8:00 PM, Guillermo Winkler gwink...@inconcertcc.com
 wrote:

 Hi Ethan,

 I want to present the events ordered by time, always in pages of 20/40
 events. If the events are tweets, you can have 1000 tweets from the same
 second or you can have 30 tweets in a 10 minute range. But I always wanna be
 able to page through the results in an orderly fashion.

 I think that using seconds since epoch it's what I'm doing, that is divide
 time into a fixed series of interval. Each second is an interval, and all of
 the events for that particular second are columns of that row.

 Again with tweets for easier visualizatoin

 TweetsBySecond : {
  12121121212 :{ - seconds since epoch
  id1,id2,id3 - all the tweet ids ocurred in that particular second
 },
 12121212123 : {
 id4,id5
 },
 12121212124 : {
 id6
 }
 }

 The problem is you can't do that using OPP in cassandra 0.7, or it's just
 me missing something?

 Thanks for your answer,
 Guille

 On Thu, Apr 14, 2011 at 4:49 PM, Ethan Rowe et...@the-rowes.com wrote:

 How do you plan to read the data?  Entire histories, or in relatively
 confined slices of time?  Do the events have any attributes by which you
 might segregate them, apart from time?

 If you can divide time into a fixed series of intervals, you can insert
 members of a given interval as columns (or supercolumns) in a row.  But it
 depends how you want to use the data on the read side.


 On Thu, Apr 14, 2011 at 12:25 PM, Guillermo Winkler 
 gwink...@inconcertcc.com wrote:

 I have a huge number of events I need to consume later, ordered by the
 date the event occured.

 My first approach to this problem was to use seconds since epoch as row
 key, and event ids as column names (empty value), this way:

 EventsByDate : {
 SecondsSinceEpoch: {
 evid:, evid:, evid:
 }
 }

 And use OPP as partitioner. Using GetRangeSlices to retrieve ordered
 events secuentially.

 Now I have two problems to solve:

 1) The system is realtime, so all the events in a given moment are
 hitting the same box
 2) Migrating from cassandra 0.6 to cassandra 0.7 OPP doesn't seem to like
 LongType for row keys, was this purposedly deprecated?

 I was thinking about secondary indexes, but it does not assure the order
 the rows are coming out of cassandra.

 Anyone has a better approach to model events by date given that
 restrictions?

 Thanks,
 Guille







Re: What's the best modeling approach for ordering events by date?

2011-04-14 Thread Ethan Rowe
How do you plan to read the data?  Entire histories, or in relatively
confined slices of time?  Do the events have any attributes by which you
might segregate them, apart from time?

If you can divide time into a fixed series of intervals, you can insert
members of a given interval as columns (or supercolumns) in a row.  But it
depends how you want to use the data on the read side.

On Thu, Apr 14, 2011 at 12:25 PM, Guillermo Winkler 
gwink...@inconcertcc.com wrote:

 I have a huge number of events I need to consume later, ordered by the date
 the event occured.

 My first approach to this problem was to use seconds since epoch as row
 key, and event ids as column names (empty value), this way:

 EventsByDate : {
 SecondsSinceEpoch: {
 evid:, evid:, evid:
 }
 }

 And use OPP as partitioner. Using GetRangeSlices to retrieve ordered events
 secuentially.

 Now I have two problems to solve:

 1) The system is realtime, so all the events in a given moment are hitting
 the same box
 2) Migrating from cassandra 0.6 to cassandra 0.7 OPP doesn't seem to like
 LongType for row keys, was this purposedly deprecated?

 I was thinking about secondary indexes, but it does not assure the order
 the rows are coming out of cassandra.

 Anyone has a better approach to model events by date given that
 restrictions?

 Thanks,
 Guille





hadoop streaming input

2011-03-17 Thread Ethan Rowe
Hello.

What's the current thinking on input support for Hadoop streaming?  It seems
like the relevant Jira issue has been quiet for some time:
https://issues.apache.org/jira/browse/CASSANDRA-1497

Thanks.
- Ethan


Re: hadoop streaming input

2011-03-17 Thread Ethan Rowe
Thanks, Jeremy.  I looked over the work that was done and it seemed like it
was mostly there, though some comments in the ticket indicated possible
problems.

I may well need to take a crack at this sometime in the next few weeks, but
if somebody beats me to it, I certainly won't complain.

On Thu, Mar 17, 2011 at 2:06 PM, Jeremy Hanna jeremy.hanna1...@gmail.comwrote:

 I started it and added the tentative patch at the end of October.  It needs
 to be rebased with the current 0.7-branch and completed - it's mostly there.
  I just tried to abstract some things in the process.

 I have changed jobs since then and I just haven't had time with the things
 I've been doing here.  If you'd like to take a stab at it, you're welcome to
 rebase and get it finished.

 On Mar 17, 2011, at 12:57 PM, Ethan Rowe wrote:

  Hello.
 
  What's the current thinking on input support for Hadoop streaming?  It
 seems like the relevant Jira issue has been quiet for some time:
  https://issues.apache.org/jira/browse/CASSANDRA-1497
 
  Thanks.
  - Ethan




Re: ORM in Cassandra?

2010-04-26 Thread Ethan Rowe

On 04/26/2010 01:26 PM, Isaac Arias wrote:

On Apr 26, 2010, at 12:13 PM, Geoffry Roberts wrote:

   

Clearly Cassandra is not an RDBMS.  The intent of my Hibernate
reference was to be more lyrical.  Sorry if that didn't come through.
 
   

Nonetheless, the need remains to relieve ourselves from excessive
boilerplate coding.
 

I agree with eliminating boilerplate code. Chris Shorrock wrote a
simple object mapper in Scala for his Cascal Cassandra client. You may
want to check out the wiki on GitHub
(http://wiki.github.com/shorrockin/cascal/).

In my opinion, a mapping solution for Cassandra should be more like a
Template. Something that helps map (back and forth) rows to objects,
columns to properties, etc. Since the data model can vary so much
depending on data access patters, any overly structured approach that
prescribes a particular schema will be of limited use.
   


For what it's worth, this is exactly my opinion after looking at the 
problem for a bit, and I'm actively developing such a solution in Ruby.  
I spent some time playing with the CassandraObject project, but felt 
that despite all the good work that went in there, it didn't feel to me 
like it fit the problem space in an idiomatic manner.  No criticism 
intended there; it seems to lean a little more towards a very structured 
schema, with less flexibility for things like collection attributes the 
members of which all have a key that matches a pattern (which is a use 
case we have).


So, for my approach, there's one project that gives metaprogramming 
semantics for building the mapping behavior you describe: build classes 
that are oriented towards mapping between simple JSON-like structures 
and full-blown business objects.  And a separate project that layers 
Cassandra specifics on top of that underlying mapper tool.


The rub being: it's for a client, and we're collectively sorting out the 
details for releasing the code in some useful, public manner.  But 
hopefully I'll get something useful out there for potential Ruby 
enthusiasts before too long.  Hopefully a week or two.


Thanks.
- Ethan

--
Ethan Rowe
End Point Corporation
et...@endpoint.com



Re: ORM in Cassandra?

2010-04-26 Thread Ethan Rowe

On 04/26/2010 03:11 PM, Tatu Saloranta wrote:

On Mon, Apr 26, 2010 at 10:35 AM, Ethan Roweet...@endpoint.com  wrote:


On 04/26/2010 01:26 PM, Isaac Arias wrote:


On Apr 26, 2010, at 12:13 PM, Geoffry Roberts wrote:



...


In my opinion, a mapping solution for Cassandra should be more like a
Template. Something that helps map (back and forth) rows to objects,
columns to properties, etc. Since the data model can vary so much
depending on data access patters, any overly structured approach that
prescribes a particular schema will be of limited use.



For what it's worth, this is exactly my opinion after looking at the problem
for a bit, and I'm actively developing such a solution in Ruby.  I spent


...


So, for my approach, there's one project that gives metaprogramming
semantics for building the mapping behavior you describe: build classes that
are oriented towards mapping between simple JSON-like structures and
full-blown business objects.  And a separate project that layers Cassandra
specifics on top of that underlying mapper tool.


+1

I think proper layering is the way to go: it makes problem (of simple
construction of services that use Cassandra as the storage system)
much easier to solve, divide and conquer. There are pretty decent
OJM/OXM solutions that are mostly orthogonal wrt distributed storage
part. I understand that there are some trade-offs (some things are
easiest to optimize when Cassandra core handles them), but flexibility
and best-tool-for-the-job have their benefits too.



Right.  Additionally, this mapping layer between simple (i.e. 
JSON-ready) structures and complex (i.e. business objects) would seem 
to be of much more general value than a Cassandra-specific mapper.  I 
would think most any environment with a heavy reliance on Thrift 
services would benefit from such tools.


--
Ethan Rowe
End Point Corporation
et...@endpoint.com



Re: Separate disks with cloud deployment

2010-03-25 Thread Ethan Rowe

On 03/25/2010 11:10 AM, Mark Greene wrote:
The FAQ page makes mention of using separate disks for the commit log 
and data directory. How would one go about achieving this in a cloud 
deployment such as Rackspace cloud servers or EC2 EBS? Or is it 
just preferred to use dedicated hardware to get the optimal performance?


With EC2, you can mount more than one EBS device on a given server, so 
it's not a big deal.  Mount one volume for logs, another volume for data.


Additionally, we've found some benefit from running (for Postgres) RAID0 
arrays on EBS; you get better I/O throughput.


I'll defer to the Rackspace folks regarding Rackspace Cloud; it has been 
I/O on average since you're dealing with a real, local disk.  But I 
don't know about getting a second disk in that environment, though.


--
Ethan Rowe
End Point Corporation
et...@endpoint.com



Re: Separate disks with cloud deployment

2010-03-25 Thread Ethan Rowe

On 03/25/2010 11:18 AM, Ethan Rowe wrote:

[snip]
I'll defer to the Rackspace folks regarding Rackspace Cloud; it has 
been I/O on average since you're dealing with a real, local disk.  But 
I don't know about getting a second disk in that environment, though.


That should have said better I/O on average.

--
Ethan Rowe
End Point Corporation
et...@endpoint.com