cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jbel...@apache.org
Subject [5/8] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Date Tue, 18 Feb 2014 17:37:02 GMT
Merge branch 'cassandra-2.0' into cassandra-2.1


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

Branch: refs/heads/trunk
Commit: 9416baa5936ca3efa28b80ebcb9632ea5ae1109e
Parents: f628bd8 f30b772
Author: Jonathan Ellis <jbellis@apache.org>
Authored: Tue Feb 18 11:36:06 2014 -0600
Committer: Jonathan Ellis <jbellis@apache.org>
Committed: Tue Feb 18 11:36:44 2014 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/service/ActiveRepairService.java  | 36 +++++++++++++++++--
 .../cassandra/service/StorageService.java       | 17 ++++-----
 .../cassandra/service/StorageServiceMBean.java  |  4 +--
 .../org/apache/cassandra/tools/NodeProbe.java   | 16 ++++-----
 .../org/apache/cassandra/tools/NodeTool.java    | 10 ++++--
 .../service/AntiEntropyServiceTestAbstract.java | 37 +++++++++++++++++---
 7 files changed, 95 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9416baa5/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index ee98f5d,fd1062e..a164534
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,42 -1,5 +1,43 @@@
 +2.1.0-beta1
 + * Add flush directory distinct from compaction directories (CASSANDRA-6357)
 + * Require JNA by default (CASSANDRA-6575)
 + * add listsnapshots command to nodetool (CASSANDRA-5742)
 + * Introduce AtomicBTreeColumns (CASSANDRA-6271, 6692)
 + * Multithreaded commitlog (CASSANDRA-3578)
 + * allocate fixed index summary memory pool and resample cold index summaries 
 +   to use less memory (CASSANDRA-5519)
 + * Removed multithreaded compaction (CASSANDRA-6142)
 + * Parallelize fetching rows for low-cardinality indexes (CASSANDRA-1337)
 + * change logging from log4j to logback (CASSANDRA-5883)
 + * switch to LZ4 compression for internode communication (CASSANDRA-5887)
 + * Stop using Thrift-generated Index* classes internally (CASSANDRA-5971)
 + * Remove 1.2 network compatibility code (CASSANDRA-5960)
 + * Remove leveled json manifest migration code (CASSANDRA-5996)
 + * Remove CFDefinition (CASSANDRA-6253)
 + * Use AtomicIntegerFieldUpdater in RefCountedMemory (CASSANDRA-6278)
 + * User-defined types for CQL3 (CASSANDRA-5590)
 + * Use of o.a.c.metrics in nodetool (CASSANDRA-5871, 6406)
 + * Batch read from OTC's queue and cleanup (CASSANDRA-1632)
 + * Secondary index support for collections (CASSANDRA-4511, 6383)
 + * SSTable metadata(Stats.db) format change (CASSANDRA-6356)
 + * Push composites support in the storage engine
 +   (CASSANDRA-5417, CASSANDRA-6520)
 + * Add snapshot space used to cfstats (CASSANDRA-6231)
 + * Add cardinality estimator for key count estimation (CASSANDRA-5906)
 + * CF id is changed to be non-deterministic. Data dir/key cache are created
 +   uniquely for CF id (CASSANDRA-5202)
 + * New counters implementation (CASSANDRA-6504)
 + * Replace UnsortedColumns, EmptyColumns, TreeMapBackedSortedColumns with new
 +   ArrayBackedSortedColumns (CASSANDRA-6630, CASSANDRA-6662, CASSANDRA-6690)
 + * Add option to use row cache with a given amount of rows (CASSANDRA-5357)
 + * Avoid repairing already repaired data (CASSANDRA-5351)
 + * Reject counter updates with USING TTL/TIMESTAMP (CASSANDRA-6649)
 + * Replace index_interval with min/max_index_interval (CASSANDRA-6379)
 + * Lift limitation that order by columns must be selected for IN queries (CASSANDRA-4911)
 +
 +
  2.0.6
+  * Allow repairing between specific replicas (CASSANDRA-6440)
   * Allow per-dc enabling of hints (CASSANDRA-6157)
   * Add compatibility for Hadoop 0.2.x (CASSANDRA-5201)
   * Fix EstimatedHistogram races (CASSANDRA-6682)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9416baa5/src/java/org/apache/cassandra/service/ActiveRepairService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/ActiveRepairService.java
index 232974d,00e43ea..83d8727
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@@ -17,9 -17,8 +17,10 @@@
   */
  package org.apache.cassandra.service;
  
 +import java.io.File;
 +import java.io.IOException;
  import java.net.InetAddress;
+ import java.net.UnknownHostException;
  import java.util.*;
  import java.util.concurrent.*;
  
@@@ -151,9 -128,7 +152,9 @@@ public class ActiveRepairServic
      // add it to the sessions (avoid NPE in tests)
      RepairFuture submitArtificialRepairSession(RepairJobDesc desc)
      {
 -        RepairSession session = new RepairSession(desc.sessionId, desc.range, desc.keyspace,
false, null, null, new String[]{desc.columnFamily});
 +        Set<InetAddress> neighbours = new HashSet<>();
-         neighbours.addAll(ActiveRepairService.getNeighbors(desc.keyspace, desc.range, null));
++        neighbours.addAll(ActiveRepairService.getNeighbors(desc.keyspace, desc.range, null,
null));
 +        RepairSession session = new RepairSession(desc.parentSessionId, desc.sessionId,
desc.range, desc.keyspace, false, neighbours, new String[]{desc.columnFamily});
          sessions.put(session.getId(), session);
          RepairFuture futureTask = new RepairFuture(session);
          executor.execute(futureTask);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9416baa5/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 91b4a36,4be95b2..31b609f
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2428,14 -2418,13 +2428,14 @@@ public class StorageService extends Not
          sendNotification(jmxNotification);
      }
  
-     public int forceRepairAsync(final String keyspace, final boolean isSequential, final
Collection<String> dataCenters, final boolean primaryRange, final boolean fullRepair,
final String... columnFamilies) throws IOException
 -    public int forceRepairAsync(final String keyspace, final boolean isSequential, final
Collection<String> dataCenters, final Collection<String> hosts, final boolean
primaryRange, final String... columnFamilies)
++    public int forceRepairAsync(final String keyspace, final boolean isSequential, final
Collection<String> dataCenters, Collection<String> hosts, final boolean primaryRange,
final boolean fullRepair, final String... columnFamilies) throws IOException
      {
          final Collection<Range<Token>> ranges = primaryRange ? getLocalPrimaryRanges(keyspace)
: getLocalRanges(keyspace);
 -        return forceRepairAsync(keyspace, isSequential, dataCenters, hosts, ranges, columnFamilies);
 +
-         return forceRepairAsync(keyspace, isSequential, dataCenters, ranges, fullRepair,
columnFamilies);
++        return forceRepairAsync(keyspace, isSequential, dataCenters, hosts, ranges, fullRepair,
columnFamilies);
      }
  
-     public int forceRepairAsync(final String keyspace, final boolean isSequential, final
Collection<String> dataCenters, final Collection<Range<Token>> ranges, final
boolean fullRepair, final String... columnFamilies) throws IOException
 -    public int forceRepairAsync(final String keyspace, final boolean isSequential, final
Collection<String> dataCenters, final Collection<String> hosts,  final Collection<Range<Token>>
ranges, final String... columnFamilies)
++    public int forceRepairAsync(final String keyspace, final boolean isSequential, final
Collection<String> dataCenters, Collection<String> hosts, final Collection<Range<Token>>
ranges, final boolean fullRepair, final String... columnFamilies) throws IOException
      {
          if (Keyspace.SYSTEM_KS.equals(keyspace) || ranges.isEmpty())
              return 0;
@@@ -2443,7 -2432,7 +2443,7 @@@
          final int cmd = nextRepairCommand.incrementAndGet();
          if (ranges.size() > 0)
          {
-             new Thread(createRepairTask(cmd, keyspace, ranges, isSequential, dataCenters,
fullRepair, columnFamilies)).start();
 -            new Thread(createRepairTask(cmd, keyspace, ranges, isSequential, dataCenters,
hosts, columnFamilies)).start();
++            new Thread(createRepairTask(cmd, keyspace, ranges, isSequential, dataCenters,
hosts, fullRepair, columnFamilies)).start();
          }
          return cmd;
      }
@@@ -2467,17 -2456,17 +2467,17 @@@
          return cmd;
      }
  
-     public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName,
boolean isSequential, Collection<String> dataCenters, boolean fullRepair, final String...
columnFamilies) throws IOException
 -    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName,
boolean isSequential, Collection<String> dataCenters, final Collection<String>
hosts, final String... columnFamilies)
++    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName,
boolean isSequential, Collection<String> dataCenters, Collection<String> hosts,
boolean fullRepair, final String... columnFamilies) throws IOException
      {
          Token parsedBeginToken = getPartitioner().getTokenFactory().fromString(beginToken);
          Token parsedEndToken = getPartitioner().getTokenFactory().fromString(endToken);
  
          logger.info("starting user-requested repair of range ({}, {}] for keyspace {} and
column families {}",
                      parsedBeginToken, parsedEndToken, keyspaceName, columnFamilies);
-         return forceRepairAsync(keyspaceName, isSequential, dataCenters, Collections.singleton(new
Range<Token>(parsedBeginToken, parsedEndToken)), fullRepair, columnFamilies);
 -        return forceRepairAsync(keyspaceName, isSequential, dataCenters, hosts, Collections.singleton(new
Range<Token>(parsedBeginToken, parsedEndToken)), columnFamilies);
++        return forceRepairAsync(keyspaceName, isSequential, dataCenters, hosts, Collections.singleton(new
Range<Token>(parsedBeginToken, parsedEndToken)), fullRepair, columnFamilies);
      }
  
 -    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName,
boolean isSequential, boolean isLocal, final String... columnFamilies)
 +    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName,
boolean isSequential, boolean isLocal, boolean fullRepair, final String... columnFamilies)
      {
          Token parsedBeginToken = getPartitioner().getTokenFactory().fromString(beginToken);
          Token parsedEndToken = getPartitioner().getTokenFactory().fromString(endToken);
@@@ -2534,18 -2517,12 +2534,19 @@@
          {
              dataCenters = Sets.newHashSet(DatabaseDescriptor.getLocalDataCenter());
          }
-         return createRepairTask(cmd, keyspace, ranges, isSequential, dataCenters, fullRepair,
columnFamilies);
 -        return createRepairTask(cmd, keyspace, ranges, isSequential, dataCenters, null,
columnFamilies);
++        return createRepairTask(cmd, keyspace, ranges, isSequential, dataCenters, null,
fullRepair, columnFamilies);
      }
  
 -    private FutureTask<Object> createRepairTask(final int cmd, final String keyspace,
final Collection<Range<Token>> ranges, final boolean isSequential, final Collection<String>
dataCenters, final Collection<String> hosts, final String... columnFamilies)
 +    private FutureTask<Object> createRepairTask(final int cmd,
 +                                                final String keyspace,
 +                                                final Collection<Range<Token>>
ranges,
 +                                                final boolean isSequential,
 +                                                final Collection<String> dataCenters,
++                                                final Collection<String> hosts,
 +                                                final boolean fullRepair,
 +                                                final String... columnFamilies)
      {
 -        return new FutureTask<Object>(new WrappedRunnable()
 +        return new FutureTask<>(new WrappedRunnable()
          {
              protected void runMayThrow() throws Exception
              {
@@@ -2568,19 -2538,7 +2569,19 @@@
                      return;
                  }
  
 -                List<RepairFuture> futures = new ArrayList<RepairFuture>(ranges.size());
 +                Set<InetAddress> neighbours = new HashSet<>();
 +                for (Range<Token> range : ranges)
-                     neighbours.addAll(ActiveRepairService.getNeighbors(keyspace, range,
dataCenters));
++                    neighbours.addAll(ActiveRepairService.getNeighbors(keyspace, range,
dataCenters, hosts));
 +
 +                List<ColumnFamilyStore> columnFamilyStores = new ArrayList<>();
 +                for (ColumnFamilyStore cfs : getValidColumnFamilies(false, false, keyspace,
columnFamilies))
 +                    columnFamilyStores.add(cfs);
 +
 +                UUID parentSession = null;
 +                if (!fullRepair)
 +                    parentSession = ActiveRepairService.instance.prepareForRepair(neighbours,
ranges, columnFamilyStores);
 +
 +                List<RepairFuture> futures = new ArrayList<>(ranges.size());
                  for (Range<Token> range : ranges)
                  {
                      RepairFuture future;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9416baa5/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 66afaa1,ed260b8..a036ba7
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -270,12 -257,12 +270,12 @@@ public interface StorageServiceMBean ex
       *
       * @return Repair command number, or 0 if nothing to repair
       */
-     public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String>
dataCenters, boolean primaryRange, boolean repairedAt, String... columnFamilies) throws IOException;
 -    public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String>
dataCenters, final Collection<String> hosts, boolean primaryRange, String... columnFamilies);
++    public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String>
dataCenters, Collection<String> hosts,  boolean primaryRange, boolean repairedAt, String...
columnFamilies) throws IOException;
  
      /**
       * Same as forceRepairAsync, but handles a specified range
       */
-     public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName,
boolean isSequential, Collection<String> dataCenters, boolean repairedAt, String...
columnFamilies) throws IOException;
 -    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName,
boolean isSequential, Collection<String> dataCenters, final Collection<String>
hosts,  final String... columnFamilies);
++    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName,
boolean isSequential, Collection<String> dataCenters, Collection<String> hosts,
boolean repairedAt, String... columnFamilies) throws IOException;
  
  
      /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9416baa5/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index 78da62a,594b41b..2288953
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -211,19 -210,19 +211,19 @@@ public class NodeProbe implements AutoC
          ssProxy.forceKeyspaceFlush(keyspaceName, columnFamilies);
      }
  
 -    public void forceKeyspaceRepair(String keyspaceName, boolean isSequential, boolean isLocal,
String... columnFamilies) throws IOException
 +    public void forceKeyspaceRepair(String keyspaceName, boolean isSequential, boolean isLocal,
boolean fullRepair, String... columnFamilies) throws IOException
      {
 -        ssProxy.forceKeyspaceRepair(keyspaceName, isSequential, isLocal, columnFamilies);
 +        ssProxy.forceKeyspaceRepair(keyspaceName, isSequential, isLocal, fullRepair, columnFamilies);
      }
  
-     public void forceRepairAsync(final PrintStream out, final String keyspaceName, boolean
isSequential, Collection<String> dataCenters, boolean primaryRange, boolean fullRepair,
String... columnFamilies) throws IOException
 -    public void forceRepairAsync(final PrintStream out, final String keyspaceName, boolean
isSequential, Collection<String> dataCenters, final Collection<String> hosts,
 boolean primaryRange, String... columnFamilies) throws IOException
++    public void forceRepairAsync(final PrintStream out, final String keyspaceName, boolean
isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean
primaryRange, boolean fullRepair, String... columnFamilies) throws IOException
      {
          RepairRunner runner = new RepairRunner(out, keyspaceName, columnFamilies);
          try
          {
              jmxc.addConnectionNotificationListener(runner, null, null);
              ssProxy.addNotificationListener(runner, null, null);
-             if (!runner.repairAndWait(ssProxy, isSequential, dataCenters, primaryRange,
fullRepair))
 -            if (!runner.repairAndWait(ssProxy, isSequential, dataCenters, hosts, primaryRange))
++            if (!runner.repairAndWait(ssProxy, isSequential, dataCenters, hosts, primaryRange,
fullRepair))
                  failed = true;
          }
          catch (Exception e)
@@@ -244,14 -240,14 +244,14 @@@
          }
      }
  
-     public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName,
boolean isSequential, Collection<String> dataCenters, final String startToken, final
String endToken, boolean fullRepair, String... columnFamilies) throws IOException
 -    public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName,
boolean isSequential, Collection<String> dataCenters, final Collection<String>
hosts, final String startToken, final String endToken, String... columnFamilies) throws IOException
++    public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName,
boolean isSequential, Collection<String> dataCenters, Collection<String> hosts,
final String startToken, final String endToken, boolean fullRepair, String... columnFamilies)
throws IOException
      {
          RepairRunner runner = new RepairRunner(out, keyspaceName, columnFamilies);
          try
          {
              jmxc.addConnectionNotificationListener(runner, null, null);
              ssProxy.addNotificationListener(runner, null, null);
-             if (!runner.repairRangeAndWait(ssProxy,  isSequential, dataCenters, startToken,
endToken, fullRepair))
 -            if (!runner.repairRangeAndWait(ssProxy,  isSequential, dataCenters, hosts, startToken,
endToken))
++            if (!runner.repairRangeAndWait(ssProxy,  isSequential, dataCenters, hosts, startToken,
endToken, fullRepair))
                  failed = true;
          }
          catch (Exception e)
@@@ -1242,16 -1045,16 +1242,16 @@@ class RepairRunner implements Notificat
          this.columnFamilies = columnFamilies;
      }
  
-     public boolean repairAndWait(StorageServiceMBean ssProxy, boolean isSequential, Collection<String>
dataCenters, boolean primaryRangeOnly, boolean fullRepair) throws Exception
 -    public boolean repairAndWait(StorageServiceMBean ssProxy, boolean isSequential, Collection<String>
dataCenters, final Collection<String> hosts, boolean primaryRangeOnly) throws Exception
++    public boolean repairAndWait(StorageServiceMBean ssProxy, boolean isSequential, Collection<String>
dataCenters, Collection<String> hosts, boolean primaryRangeOnly, boolean fullRepair)
throws Exception
      {
-         cmd = ssProxy.forceRepairAsync(keyspace, isSequential, dataCenters, primaryRangeOnly,
fullRepair, columnFamilies);
 -        cmd = ssProxy.forceRepairAsync(keyspace, isSequential, dataCenters, hosts, primaryRangeOnly,
columnFamilies);
++        cmd = ssProxy.forceRepairAsync(keyspace, isSequential, dataCenters, hosts, primaryRangeOnly,
fullRepair, columnFamilies);
          waitForRepair();
          return success;
      }
  
-     public boolean repairRangeAndWait(StorageServiceMBean ssProxy, boolean isSequential,
Collection<String> dataCenters, String startToken, String endToken, boolean fullRepair)
throws Exception
 -    public boolean repairRangeAndWait(StorageServiceMBean ssProxy, boolean isSequential,
Collection<String> dataCenters, final Collection<String> hosts, String startToken,
String endToken) throws Exception
++    public boolean repairRangeAndWait(StorageServiceMBean ssProxy, boolean isSequential,
Collection<String> dataCenters, Collection<String> hosts, String startToken, String
endToken, boolean fullRepair) throws Exception
      {
-         cmd = ssProxy.forceRepairRangeAsync(startToken, endToken, keyspace, isSequential,
dataCenters, fullRepair, columnFamilies);
 -        cmd = ssProxy.forceRepairRangeAsync(startToken, endToken, keyspace, isSequential,
dataCenters, hosts, columnFamilies);
++        cmd = ssProxy.forceRepairRangeAsync(startToken, endToken, keyspace, isSequential,
dataCenters, hosts, fullRepair, columnFamilies);
          waitForRepair();
          return success;
      }


Mime
View raw message