You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by yu...@apache.org on 2015/12/01 20:08:33 UTC

[10/15] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Merge branch 'cassandra-2.2' into cassandra-3.0


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

Branch: refs/heads/trunk
Commit: ccb20ad46ab38961aac39cc8634f450046bdf16b
Parents: 803a3d9 2491ede
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue Dec 1 13:07:39 2015 -0600
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Dec 1 13:07:39 2015 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/streaming/StreamReceiveTask.java  | 60 ++++++++++----------
 2 files changed, 31 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ccb20ad4/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 7fffbbf,7541212..a01011b
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -23,7 -8,16 +23,8 @@@ Merged from 2.2
   * Fix SimpleDateType type compatibility (CASSANDRA-10027)
   * (Hadoop) fix splits calculation (CASSANDRA-10640)
   * (Hadoop) ensure that Cluster instances are always closed (CASSANDRA-10058)
 - * (cqlsh) show partial trace if incomplete after max_trace_wait (CASSANDRA-7645)
 - * Use most up-to-date version of schema for system tables (CASSANDRA-10652)
 - * Deprecate memory_allocator in cassandra.yaml (CASSANDRA-10581,10628)
 - * Expose phi values from failure detector via JMX and tweak debug
 -   and trace logging (CASSANDRA-9526)
 - * Fix RangeNamesQueryPager (CASSANDRA-10509)
 - * Deprecate Pig support (CASSANDRA-10542)
 - * Reduce contention getting instances of CompositeType (CASSANDRA-10433)
  Merged from 2.1:
+  * Add proper error handling to stream receiver (CASSANDRA-10774)
   * Warn or fail when changing cluster topology live (CASSANDRA-10243)
   * Status command in debian/ubuntu init script doesn't work (CASSANDRA-10213)
   * Some DROP ... IF EXISTS incorrectly result in exceptions on non-existing KS (CASSANDRA-10658)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ccb20ad4/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
index 54ce711,dd56b8b..dfc91f9
--- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@@ -126,110 -113,73 +126,110 @@@ public class StreamReceiveTask extends 
  
          public void run()
          {
-             Pair<String, String> kscf = Schema.instance.getCF(task.cfId);
-             if (kscf == null)
-             {
-                 // schema was dropped during streaming
-                 task.sstables.forEach(SSTableMultiWriter::abortOrDie);
- 
-                 task.sstables.clear();
-                 task.txn.abort();
-                 return;
-             }
-             ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
-             boolean hasViews = !Iterables.isEmpty(View.findAll(kscf.left, kscf.right));
- 
++            boolean hasViews = false;
++            ColumnFamilyStore cfs = null;
              try
              {
+                 Pair<String, String> kscf = Schema.instance.getCF(task.cfId);
+                 if (kscf == null)
+                 {
+                     // schema was dropped during streaming
 -                    for (SSTableWriter writer : task.sstables)
 -                        writer.abort();
++                    task.sstables.forEach(SSTableMultiWriter::abortOrDie);
+                     task.sstables.clear();
++                    task.txn.abort();
+                     task.session.taskCompleted(task);
+                     return;
+                 }
 -                ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
++                cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
++                hasViews = !Iterables.isEmpty(View.findAll(kscf.left, kscf.right));
+ 
 -                File lockfiledir = cfs.directories.getWriteableLocationAsFile(task.sstables.size() * 256L);
 -                if (lockfiledir == null)
 -                    throw new IOError(new IOException("All disks full"));
 -                StreamLockfile lockfile = new StreamLockfile(lockfiledir, UUID.randomUUID());
 -                lockfile.create(task.sstables);
                  List<SSTableReader> readers = new ArrayList<>();
 -                for (SSTableWriter writer : task.sstables)
 -                    readers.add(writer.finish(true));
 -                lockfile.delete();
 +                for (SSTableMultiWriter writer : task.sstables)
 +                {
 +                    Collection<SSTableReader> newReaders = writer.finish(true);
 +                    readers.addAll(newReaders);
 +                    task.txn.update(newReaders, false);
 +                }
 +
                  task.sstables.clear();
  
                  try (Refs<SSTableReader> refs = Refs.ref(readers))
                  {
 -                    // add sstables and build secondary indexes
 -                    cfs.addSSTables(readers);
 -                    cfs.indexManager.maybeBuildSecondaryIndexes(readers, cfs.indexManager.allIndexesNames());
 -
 -                    //invalidate row and counter cache
 -                    if (cfs.isRowCacheEnabled() || cfs.metadata.isCounter())
 +                    //We have a special path for views.
 +                    //Since the view requires cleaning up any pre-existing state, we must put
 +                    //all partitions through the same write path as normal mutations.
 +                    //This also ensures any 2is are also updated
 +                    if (hasViews)
                      {
 -                        List<Bounds<Token>> boundsToInvalidate = new ArrayList<>(readers.size());
 -                        for (SSTableReader sstable : readers)
 -                            boundsToInvalidate.add(new Bounds<Token>(sstable.first.getToken(), sstable.last.getToken()));
 -                        Set<Bounds<Token>> nonOverlappingBounds = Bounds.getNonOverlappingBounds(boundsToInvalidate);
 -
 -                        if (cfs.isRowCacheEnabled())
 +                        for (SSTableReader reader : readers)
                          {
 -                            int invalidatedKeys = cfs.invalidateRowCache(nonOverlappingBounds);
 -                            if (invalidatedKeys > 0)
 -                                logger.debug("[Stream #{}] Invalidated {} row cache entries on table {}.{} after stream " +
 -                                             "receive task completed.", task.session.planId(), invalidatedKeys,
 -                                             cfs.keyspace.getName(), cfs.getColumnFamilyName());
 +                            try (ISSTableScanner scanner = reader.getScanner())
 +                            {
 +                                while (scanner.hasNext())
 +                                {
 +                                    try (UnfilteredRowIterator rowIterator = scanner.next())
 +                                    {
 +                                        //Apply unsafe (we will flush below before transaction is done)
 +                                        new Mutation(PartitionUpdate.fromIterator(rowIterator)).applyUnsafe();
 +                                    }
 +                                }
 +                            }
                          }
 +                    }
 +                    else
 +                    {
 +                        task.txn.finish();
  
 -                        if (cfs.metadata.isCounter())
 +                        // add sstables and build secondary indexes
 +                        cfs.addSSTables(readers);
 +                        cfs.indexManager.buildAllIndexesBlocking(readers);
 +
 +                        //invalidate row and counter cache
 +                        if (cfs.isRowCacheEnabled() || cfs.metadata.isCounter())
                          {
 -                            int invalidatedKeys = cfs.invalidateCounterCache(nonOverlappingBounds);
 -                            if (invalidatedKeys > 0)
 -                                logger.debug("[Stream #{}] Invalidated {} counter cache entries on table {}.{} after stream " +
 -                                             "receive task completed.", task.session.planId(), invalidatedKeys,
 -                                             cfs.keyspace.getName(), cfs.getColumnFamilyName());
 +                            List<Bounds<Token>> boundsToInvalidate = new ArrayList<>(readers.size());
 +                            readers.forEach(sstable -> boundsToInvalidate.add(new Bounds<Token>(sstable.first.getToken(), sstable.last.getToken())));
 +                            Set<Bounds<Token>> nonOverlappingBounds = Bounds.getNonOverlappingBounds(boundsToInvalidate);
 +
 +                            if (cfs.isRowCacheEnabled())
 +                            {
 +                                int invalidatedKeys = cfs.invalidateRowCache(nonOverlappingBounds);
 +                                if (invalidatedKeys > 0)
 +                                    logger.debug("[Stream #{}] Invalidated {} row cache entries on table {}.{} after stream " +
 +                                                 "receive task completed.", task.session.planId(), invalidatedKeys,
 +                                                 cfs.keyspace.getName(), cfs.getTableName());
 +                            }
 +
 +                            if (cfs.metadata.isCounter())
 +                            {
 +                                int invalidatedKeys = cfs.invalidateCounterCache(nonOverlappingBounds);
 +                                if (invalidatedKeys > 0)
 +                                    logger.debug("[Stream #{}] Invalidated {} counter cache entries on table {}.{} after stream " +
 +                                                 "receive task completed.", task.session.planId(), invalidatedKeys,
 +                                                 cfs.keyspace.getName(), cfs.getTableName());
 +                            }
                          }
-                     }
-                 }
-                 catch (Throwable t)
-                 {
-                     logger.error("Error applying streamed sstable: ", t);
- 
-                     JVMStabilityInspector.inspectThrowable(t);
-                 }
-                 finally
-                 {
-                     //We don't keep the streamed sstables since we've applied them manually
-                     //So we abort the txn and delete the streamed sstables
-                     if (hasViews)
-                     {
-                         cfs.forceBlockingFlush();
-                         task.txn.abort();
++                        task.session.taskCompleted(task);
                      }
                  }
 -
 -                task.session.taskCompleted(task);
              }
+             catch (Throwable t)
+             {
+                 logger.error("Error applying streamed data: ", t);
+                 JVMStabilityInspector.inspectThrowable(t);
+                 task.session.onError(t);
+             }
 +            finally
 +            {
-                 task.session.taskCompleted(task);
++                //We don't keep the streamed sstables since we've applied them manually
++                //So we abort the txn and delete the streamed sstables
++                if (hasViews)
++                {
++                    if (cfs != null)
++                        cfs.forceBlockingFlush();
++                    task.txn.abort();
++                }
 +            }
          }
      }