cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From tylerho...@apache.org
Subject [2/2] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Date Thu, 18 Sep 2014 17:36:49 GMT
Merge branch 'cassandra-2.0' into cassandra-2.1

Conflicts:
	src/java/org/apache/cassandra/transport/Message.java
	src/java/org/apache/cassandra/transport/messages/ErrorMessage.java


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

Branch: refs/heads/cassandra-2.1
Commit: 3dd2f0064f49aa543647be9882dfe8da5ab6899b
Parents: 74877bb cbc705d
Author: Tyler Hobbs <tyler@datastax.com>
Authored: Thu Sep 18 12:36:33 2014 -0500
Committer: Tyler Hobbs <tyler@datastax.com>
Committed: Thu Sep 18 12:36:33 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  4 +-
 .../org/apache/cassandra/transport/Message.java | 76 +++++++++++++++++++-
 .../transport/messages/ErrorMessage.java        | 15 +++-
 3 files changed, 90 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/3dd2f006/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index d25fee5,c9d507a..dfcd36d
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,50 -1,6 +1,50 @@@
 -2.0.11:
 +2.1.1
 + * Provide visibility into prepared statements churn (CASSANDRA-7921)
 + * Invalidate prepared statements when their keyspace or table is
 +   dropped (CASSANDRA-7566)
 + * cassandra-stress: fix support for NetworkTopologyStrategy (CASSANDRA-7945)
 + * Fix saving caches when a table is dropped (CASSANDRA-7784)
 + * Add better error checking of new stress profile (CASSANDRA-7716)
 + * Use ThreadLocalRandom and remove FBUtilities.threadLocalRandom (CASSANDRA-7934)
 + * Prevent operator mistakes due to simultaneous bootstrap (CASSANDRA-7069)
 + * cassandra-stress supports whitelist mode for node config (CASSANDRA-7658)
 + * GCInspector more closely tracks GC; cassandra-stress and nodetool report it (CASSANDRA-7916)
 + * nodetool won't output bogus ownership info without a keyspace (CASSANDRA-7173)
 + * Add human readable option to nodetool commands (CASSANDRA-5433)
 + * Don't try to set repairedAt on old sstables (CASSANDRA-7913)
 + * Add metrics for tracking PreparedStatement use (CASSANDRA-7719)
 + * (cqlsh) tab-completion for triggers (CASSANDRA-7824)
 + * (cqlsh) Support for query paging (CASSANDRA-7514)
 + * (cqlsh) Show progress of COPY operations (CASSANDRA-7789)
 + * Add syntax to remove multiple elements from a map (CASSANDRA-6599)
 + * Support non-equals conditions in lightweight transactions (CASSANDRA-6839)
 + * Add IF [NOT] EXISTS to create/drop triggers (CASSANDRA-7606)
 + * (cqlsh) Display the current logged-in user (CASSANDRA-7785)
 + * (cqlsh) Don't ignore CTRL-C during COPY FROM execution (CASSANDRA-7815)
 + * (cqlsh) Order UDTs according to cross-type dependencies in DESCRIBE
 +   output (CASSANDRA-7659)
 + * (cqlsh) Fix handling of CAS statement results (CASSANDRA-7671)
 + * (cqlsh) COPY TO/FROM improvements (CASSANDRA-7405)
 + * Support list index operations with conditions (CASSANDRA-7499)
 + * Add max live/tombstoned cells to nodetool cfstats output (CASSANDRA-7731)
 + * Validate IPv6 wildcard addresses properly (CASSANDRA-7680)
 + * (cqlsh) Error when tracing query (CASSANDRA-7613)
 + * Avoid IOOBE when building SyntaxError message snippet (CASSANDRA-7569)
 + * SSTableExport uses correct validator to create string representation of partition
 +   keys (CASSANDRA-7498)
 + * Avoid NPEs when receiving type changes for an unknown keyspace (CASSANDRA-7689)
 + * Add support for custom 2i validation (CASSANDRA-7575)
 + * Pig support for hadoop CqlInputFormat (CASSANDRA-6454)
 + * Add listen_interface and rpc_interface options (CASSANDRA-7417)
 + * Improve schema merge performance (CASSANDRA-7444)
 + * Adjust MT depth based on # of partition validating (CASSANDRA-5263)
 + * Optimise NativeCell comparisons (CASSANDRA-6755)
 + * Configurable client timeout for cqlsh (CASSANDRA-7516)
 + * Include snippet of CQL query near syntax error in messages (CASSANDRA-7111)
 + * Make repair -pr work with -local (CASSANDRA-7450)
 +Merged from 2.0:
- =======
- 2.0.11:
+  * Log exceptions related to unclean native protocol client disconnects
+    at DEBUG or INFO (CASSANDRA-7849)
   * Allow permissions cache to be set via JMX (CASSANDRA-7698)
   * Include schema_triggers CF in readable system resources (CASSANDRA-7967)
   * Fix RowIndexEntry to report correct serializedSize (CASSANDRA-7948)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3dd2f006/src/java/org/apache/cassandra/transport/Message.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/transport/Message.java
index 163fffb,eb38558..f4c6b31
--- a/src/java/org/apache/cassandra/transport/Message.java
+++ b/src/java/org/apache/cassandra/transport/Message.java
@@@ -17,21 -17,18 +17,26 @@@
   */
  package org.apache.cassandra.transport;
  
 +import java.util.ArrayList;
+ import java.io.IOException;
  import java.util.EnumSet;
 +import java.util.HashSet;
 +import java.util.List;
+ import java.util.Set;
  import java.util.UUID;
 +import java.util.concurrent.ConcurrentHashMap;
 +import java.util.concurrent.ConcurrentLinkedQueue;
 +import java.util.concurrent.ConcurrentMap;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.atomic.AtomicBoolean;
 +
 +import io.netty.buffer.ByteBuf;
 +import io.netty.channel.*;
 +import io.netty.handler.codec.MessageToMessageDecoder;
 +import io.netty.handler.codec.MessageToMessageEncoder;
+ 
+ import com.google.common.base.Predicate;
+ import com.google.common.collect.ImmutableSet;
 -import org.jboss.netty.buffer.ChannelBuffer;
 -import org.jboss.netty.buffer.ChannelBuffers;
 -import org.jboss.netty.channel.*;
 -import org.jboss.netty.handler.codec.oneone.OneToOneDecoder;
 -import org.jboss.netty.handler.codec.oneone.OneToOneEncoder;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
@@@ -426,38 -325,24 +442,40 @@@ public abstract class Messag
              }
              catch (Throwable ex)
              {
-                 flush(new FlushItem(ctx, ErrorMessage.fromException(ex).setStreamId(request.getStreamId()),
request.getSourceFrame()));
 -                // Don't let the exception propagate to exceptionCaught() if we can help
it so that we can assign the right streamID.
 -                ctx.getChannel().write(ErrorMessage.fromException(ex, new UnexpectedChannelExceptionHandler(ctx.getChannel(),
true)).setStreamId(request.getStreamId()));
++                UnexpectedChannelExceptionHandler handler = new UnexpectedChannelExceptionHandler(ctx.channel(),
true);
++                flush(new FlushItem(ctx, ErrorMessage.fromException(ex, handler).setStreamId(request.getStreamId()),
request.getSourceFrame()));
 +                return;
 +            }
 +
 +            logger.debug("Responding: {}, v={}", response, connection.getVersion());
 +            flush(new FlushItem(ctx, response, request.getSourceFrame()));
 +        }
 +
 +        private void flush(FlushItem item)
 +        {
 +            EventLoop loop = item.ctx.channel().eventLoop();
 +            Flusher flusher = flusherLookup.get(loop);
 +            if (flusher == null)
 +            {
 +                Flusher alt = flusherLookup.putIfAbsent(loop, flusher = new Flusher(loop));
 +                if (alt != null)
 +                    flusher = alt;
              }
 +
 +            flusher.queued.add(item);
 +            flusher.start();
          }
  
          @Override
 -        public void exceptionCaught(final ChannelHandlerContext ctx, ExceptionEvent e)
 +        public void exceptionCaught(final ChannelHandlerContext ctx, Throwable cause)
          throws Exception
          {
 -            if (ctx.getChannel().isOpen())
 +            if (ctx.channel().isOpen())
              {
-                 ChannelFuture future = ctx.writeAndFlush(ErrorMessage.fromException(cause));
 -                ChannelFuture future = ctx.getChannel().write(ErrorMessage.fromException(e.getCause(),
new UnexpectedChannelExceptionHandler(ctx.getChannel(), false)));
++                UnexpectedChannelExceptionHandler handler = new UnexpectedChannelExceptionHandler(ctx.channel(),
false);
++                ChannelFuture future = ctx.writeAndFlush(ErrorMessage.fromException(cause,
handler));
                  // On protocol exception, close the channel as soon as the message have
been sent
 -                if (e.getCause() instanceof ProtocolException)
 +                if (cause instanceof ProtocolException)
                  {
                      future.addListener(new ChannelFutureListener() {
                          public void operationComplete(ChannelFuture future) {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3dd2f006/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
index 6109d1d,4d60a1f..0aa54f1
--- a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
@@@ -17,7 -17,8 +17,8 @@@
   */
  package org.apache.cassandra.transport.messages;
  
 +import io.netty.buffer.ByteBuf;
+ import com.google.common.base.Predicate;
 -import org.jboss.netty.buffer.ChannelBuffer;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  


Mime
View raw message