cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From alek...@apache.org
Subject [3/3] cassandra git commit: Merge branch 'cassandra-2.1' into trunk
Date Sun, 22 Mar 2015 11:35:33 GMT
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 83e6820317eb1705c72d6ea1b3ef3937897e6106
Parents: 7145b2c 6c69f9a
Author: Aleksey Yeschenko <aleksey@apache.org>
Authored: Sun Mar 22 14:35:20 2015 +0300
Committer: Aleksey Yeschenko <aleksey@apache.org>
Committed: Sun Mar 22 14:35:20 2015 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 ++
 .../cql3/statements/BatchStatement.java         |  2 +-
 .../cassandra/db/CollationController.java       |  5 ++++-
 .../cassandra/db/filter/SliceQueryFilter.java   | 23 +++++++++++++++-----
 .../filter/TombstoneOverwhelmingException.java  | 19 ++++++++--------
 .../apache/cassandra/service/StorageProxy.java  |  9 ++++----
 .../org/apache/cassandra/tracing/Tracing.java   |  2 +-
 7 files changed, 40 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/83e68203/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/83e68203/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index 59f8f27,c93bf64..84f35ff
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@@ -253,17 -239,8 +253,17 @@@ public class BatchStatement implements 
              for (ColumnFamily cf : cfs)
                  ksCfPairs.add(cf.metadata().ksName + "." + cf.metadata().cfName);
  
 -            String format = "Batch of prepared statements for {} is of size {}, exceeding
specified threshold of {} by {}.";
 -            logger.warn(format, ksCfPairs, size, warnThreshold, size - warnThreshold);
 +            String format = "Batch of prepared statements for {} is of size {}, exceeding
specified threshold of {} by {}.{}";
 +            if (size > failThreshold)
 +            {
-                 Tracing.trace(format, new Object[] {ksCfPairs, size, failThreshold, size
- failThreshold, " (see batch_size_fail_threshold_in_kb)"});
++                Tracing.trace(format, ksCfPairs, size, failThreshold, size - failThreshold,
" (see batch_size_fail_threshold_in_kb)");
 +                logger.error(format, ksCfPairs, size, failThreshold, size - failThreshold,
" (see batch_size_fail_threshold_in_kb)");
 +                throw new InvalidRequestException(String.format("Batch too large"));
 +            }
 +            else if (logger.isWarnEnabled())
 +            {
 +                logger.warn(format, ksCfPairs, size, warnThreshold, size - warnThreshold,
"");
 +            }
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/83e68203/src/java/org/apache/cassandra/db/CollationController.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/CollationController.java
index d266351,4efcd9c..ab75565
--- a/src/java/org/apache/cassandra/db/CollationController.java
+++ b/src/java/org/apache/cassandra/db/CollationController.java
@@@ -309,8 -309,8 +309,11 @@@ public class CollationControlle
                      }
                  }
              }
++
              if (Tracing.isTracing())
--                Tracing.trace("Skipped {}/{} non-slice-intersecting sstables, included {}
due to tombstones", new Object[] {nonIntersectingSSTables, view.sstables.size(), includedDueToTombstones});
++                Tracing.trace("Skipped {}/{} non-slice-intersecting sstables, included {}
due to tombstones",
++                              nonIntersectingSSTables, view.sstables.size(), includedDueToTombstones);
++
              // we need to distinguish between "there is no data at all for this row" (BF
will let us rebuild that efficiently)
              // and "there used to be data, but it's gone now" (we should cache the empty
CF so we don't need to rebuild that slower)
              if (iterators.isEmpty())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/83e68203/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
index f984f96,540aeef..8c6468b
--- a/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
@@@ -213,41 -214,34 +213,52 @@@ public class SliceQueryFilter implement
  
              if (respectTombstoneThresholds() && columnCounter.ignored() > DatabaseDescriptor.getTombstoneFailureThreshold())
              {
 -                Tracing.trace("Scanned over {} tombstones; query aborted (see tombstone_failure_threshold)",
DatabaseDescriptor.getTombstoneFailureThreshold());
 -                logger.error("Scanned over {} tombstones in {}.{}; query aborted (see tombstone_failure_threshold)",
 -                             DatabaseDescriptor.getTombstoneFailureThreshold(), container.metadata().ksName,
container.metadata().cfName);
 -                throw new TombstoneOverwhelmingException();
 +                Tracing.trace("Scanned over {} tombstones; query aborted (see tombstone_failure_threshold);
slices={}",
 +                              DatabaseDescriptor.getTombstoneFailureThreshold(), getSlicesInfo(container));
-                 throw new TombstoneOverwhelmingException(columnCounter.ignored(), count,
container.metadata().ksName, container.metadata().cfName,
-                                 container.getComparator().getString(cell.name()), getSlicesInfo(container),
 container.deletionInfo().toString());
++
++                throw new TombstoneOverwhelmingException(columnCounter.ignored(),
++                                                         count,
++                                                         container.metadata().ksName,
++                                                         container.metadata().cfName,
++                                                         container.getComparator().getString(cell.name()),
++                                                         getSlicesInfo(container));
              }
  
              container.maybeAppendColumn(cell, tester, gcBefore);
          }
  
 -        Tracing.trace("Read {} live and {} tombstoned cells", columnCounter.live(), columnCounter.ignored());
 -        if (respectTombstoneThresholds() && columnCounter.ignored() > DatabaseDescriptor.getTombstoneWarnThreshold())
 +        boolean warnTombstones = respectTombstoneThresholds() && columnCounter.ignored()
> DatabaseDescriptor.getTombstoneWarnThreshold();
 +        if (warnTombstones)
          {
-             logger.warn("Read {} live and {} tombstoned cells in {}.{} (see tombstone_warn_threshold).
{} columns were requested, slices={}, delInfo={}",
-                         columnCounter.live(), columnCounter.ignored(), container.metadata().ksName,
container.metadata().cfName, count,
-                         getSlicesInfo(container), container.deletionInfo());
 -            StringBuilder sb = new StringBuilder();
 -            CellNameType type = container.metadata().comparator;
 -            for (ColumnSlice sl : slices)
 -            {
 -                assert sl != null;
++            logger.warn("Read {} live and {} tombstoned cells in {}.{} (see tombstone_warn_threshold).
{} columns were requested, slices={}",
++                        columnCounter.live(),
++                        columnCounter.ignored(),
++                        container.metadata().ksName,
++                        container.metadata().cfName,
++                        count,
++                        getSlicesInfo(container));
 +        }
 +        Tracing.trace("Read {} live and {} tombstoned cells{}",
-                       new Object[]{ columnCounter.live(), columnCounter.ignored(), (warnTombstones
? " (see tombstone_warn_threshold)" : "") });
++                      columnCounter.live(),
++                      columnCounter.ignored(),
++                      warnTombstones ? " (see tombstone_warn_threshold)" : "");
 +    }
  
 -                sb.append('[');
 -                sb.append(type.getString(sl.start));
 -                sb.append('-');
 -                sb.append(type.getString(sl.finish));
 -                sb.append(']');
 -            }
 +    private String getSlicesInfo(ColumnFamily container)
 +    {
 +        StringBuilder sb = new StringBuilder();
 +        CellNameType type = container.metadata().comparator;
 +        for (ColumnSlice sl : slices)
 +        {
 +            assert sl != null;
  
 -            logger.warn("Read {} live and {} tombstoned cells in {}.{} (see tombstone_warn_threshold).
{} columns was requested, slices={}",
 -                        columnCounter.live(), columnCounter.ignored(), container.metadata().ksName,
container.metadata().cfName, count, sb);
 +            sb.append('[');
 +            sb.append(type.getString(sl.start));
 +            sb.append('-');
 +            sb.append(type.getString(sl.finish));
 +            sb.append(']');
          }
 +        return sb.toString();
      }
  
      protected boolean respectTombstoneThresholds()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/83e68203/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java
index 04d440d,6a6b0f6..7624e1b
--- a/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java
+++ b/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java
@@@ -18,48 -18,6 +18,49 @@@
   */
  package org.apache.cassandra.db.filter;
  
 +import org.apache.cassandra.db.DecoratedKey;
 +
- 
  public class TombstoneOverwhelmingException extends RuntimeException
  {
 +    private final int numTombstones;
 +    private final int numRequested;
 +    private final String ksName;
 +    private final String cfName;
 +    private final String lastCellName;
 +    private final String slicesInfo;
-     private final String deletionInfo;
 +    private String partitionKey = null;
 +
-     public TombstoneOverwhelmingException(int numTombstones, int numRequested, String ksName,
String cfName,
-                                           String lastCellName, String slicesInfo, String
deletionInfo)
++    public TombstoneOverwhelmingException(int numTombstones,
++                                          int numRequested,
++                                          String ksName,
++                                          String cfName,
++                                          String lastCellName,
++                                          String slicesInfo)
 +    {
 +        this.numTombstones = numTombstones;
 +        this.numRequested = numRequested;
 +        this.ksName = ksName;
 +        this.cfName = cfName;
 +        this.lastCellName = lastCellName;
 +        this.slicesInfo = slicesInfo;
-         this.deletionInfo = deletionInfo;
 +    }
 +
 +    public void setKey(DecoratedKey key)
 +    {
-         if(key != null)
-             this.partitionKey = key.toString();
++        if (key != null)
++            partitionKey = key.toString();
 +    }
 +
 +    public String getLocalizedMessage()
 +    {
 +        return getMessage();
 +    }
 +
 +    public String getMessage()
 +    {
 +        return String.format(
 +                "Scanned over %d tombstones in %s.%s; %d columns were requested; query aborted
" +
-                 "(see tombstone_failure_threshold); partitionKey=%s; lastCell=%s; delInfo=%s;
slices=%s",
-                 numTombstones, ksName, cfName, numRequested, partitionKey, lastCellName,
deletionInfo, slicesInfo);
++                "(see tombstone_failure_threshold); partitionKey=%s; lastCell=%s; slices=%s",
++                numTombstones, ksName, cfName, numRequested, partitionKey, lastCellName,
slicesInfo);
 +    }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/83e68203/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageProxy.java
index 45c4c4d,2ca94b7..5875fbc
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@@ -552,19 -567,9 +552,19 @@@ public class StorageProxy implements St
              }
              else
              {
 -                writeMetrics.timeouts.mark();
 -                ClientRequestMetrics.writeTimeouts.inc();
 -                Tracing.trace("Write timeout; received {} of {} required replies", ex.received,
ex.blockFor);
 +                if (ex instanceof WriteFailureException)
 +                {
 +                    writeMetrics.failures.mark();
 +                    WriteFailureException fe = (WriteFailureException)ex;
 +                    Tracing.trace("Write failure; received {} of {} required replies, failed
{} requests",
-                         new Object[] {fe.received, fe.blockFor, fe.failures});
++                                  fe.received, fe.blockFor, fe.failures);
 +                }
 +                else
 +                {
 +                    writeMetrics.timeouts.mark();
 +                    WriteTimeoutException te = (WriteTimeoutException)ex;
 +                    Tracing.trace("Write timeout; received {} of {} required replies", te.received,
te.blockFor);
 +                }
                  throw ex;
              }
          }
@@@ -1393,11 -1353,10 +1393,11 @@@
                                     ? exec.resolver.isDataPresent() ? " (including data)"
: " (only digests)"
                                     : "";
  
 +                    boolean isTimeout = ex instanceof ReadTimeoutException;
                      if (Tracing.isTracing())
                      {
 -                        Tracing.trace("Timed out; received {} of {} responses{}",
 -                                      new Object[]{ responseCount, blockFor, gotData });
 +                        Tracing.trace("{}; received {} of {} responses{}",
-                                       new Object[]{(isTimeout ? "Timed out" : "Failed"),
responseCount, blockFor, gotData });
++                                      isTimeout ? "Timed out" : "Failed", responseCount,
blockFor, gotData);
                      }
                      else if (logger.isDebugEnabled())
                      {
@@@ -1686,7 -1623,7 +1686,8 @@@
                                    : Math.max(1, Math.min(ranges.size(), (int) Math.ceil(command.limit()
/ resultRowsPerRange)));
              logger.debug("Estimated result rows per range: {}; requested rows: {}, ranges.size():
{}; concurrent range requests: {}",
                           resultRowsPerRange, command.limit(), ranges.size(), concurrencyFactor);
--            Tracing.trace("Submitting range requests on {} ranges with a concurrency of
{} ({} rows per range expected)", new Object[]{ ranges.size(), concurrencyFactor, resultRowsPerRange});
++            Tracing.trace("Submitting range requests on {} ranges with a concurrency of
{} ({} rows per range expected)",
++                          ranges.size(), concurrencyFactor, resultRowsPerRange);
  
              boolean haveSufficientRows = false;
              int i = 0;
@@@ -1801,11 -1738,10 +1802,11 @@@
                                           ? resolver.isDataPresent() ? " (including data)"
: " (only digests)"
                                           : "";
  
 +                        boolean isTimeout = ex instanceof ReadTimeoutException;
                          if (Tracing.isTracing())
                          {
 -                            Tracing.trace("Timed out; received {} of {} responses{} for
range {} of {}",
 -                                          new Object[]{ responseCount, blockFor, gotData,
i, ranges.size() });
 +                            Tracing.trace("{}; received {} of {} responses{} for range {}
of {}",
-                                           new Object[]{(isTimeout ? "Timed out" : "Failed"),
responseCount, blockFor, gotData, i, ranges.size() });
++                                          (isTimeout ? "Timed out" : "Failed"), responseCount,
blockFor, gotData, i, ranges.size());
                          }
                          else if (logger.isDebugEnabled())
                          {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/83e68203/src/java/org/apache/cassandra/tracing/Tracing.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tracing/Tracing.java
index 9b33444,d74859a..dc7067e
--- a/src/java/org/apache/cassandra/tracing/Tracing.java
+++ b/src/java/org/apache/cassandra/tracing/Tracing.java
@@@ -296,7 -235,7 +296,7 @@@ public class Tracin
          state.trace(format, arg1, arg2);
      }
  
--    public static void trace(String format, Object[] args)
++    public static void trace(String format, Object... args)
      {
          final TraceState state = instance.get();
          if (state == null) // inline isTracing to avoid implicit two calls to state.get()


Mime
View raw message