You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ad...@apache.org on 2021/10/26 17:24:43 UTC

[cassandra] branch cassandra-3.11 updated (fa532a6 -> c76a939)

This is an automated email from the ASF dual-hosted git repository.

adelapena pushed a change to branch cassandra-3.11
in repository https://gitbox.apache.org/repos/asf/cassandra.git.


    from fa532a6  Prevent broken concurrent schema read/writes
     new 7f54fe0  Fix failure handling in inter-node communication
     new c76a939  Merge branch 'cassandra-3.0' into cassandra-3.11

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 CHANGES.txt                                           |  1 +
 .../org/apache/cassandra/db/MutationVerbHandler.java  |  3 ++-
 .../org/apache/cassandra/net/MessageDeliveryTask.java | 19 ++++++++++++++++---
 .../service/AbstractWriteResponseHandler.java         |  3 +++
 .../org/apache/cassandra/service/StorageProxy.java    |  2 +-
 5 files changed, 23 insertions(+), 5 deletions(-)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[cassandra] 01/01: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

adelapena pushed a commit to branch cassandra-3.11
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit c76a939c3eb9aa68abd0b892ab09bcbf09157e10
Merge: fa532a6 7f54fe0
Author: Andrés de la Peña <a....@gmail.com>
AuthorDate: Tue Oct 26 18:18:01 2021 +0100

    Merge branch 'cassandra-3.0' into cassandra-3.11

 CHANGES.txt                                           |  1 +
 .../org/apache/cassandra/db/MutationVerbHandler.java  |  3 ++-
 .../org/apache/cassandra/net/MessageDeliveryTask.java | 19 ++++++++++++++++---
 .../service/AbstractWriteResponseHandler.java         |  3 +++
 .../org/apache/cassandra/service/StorageProxy.java    |  2 +-
 5 files changed, 23 insertions(+), 5 deletions(-)

diff --cc CHANGES.txt
index 8460705,7dd58ff..0f8b5de
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,16 -1,5 +1,17 @@@
 -3.0.26:
 +3.11.12
 + * Upgrade snakeyaml to 1.26 in 3.11 (CASSANDRA=17028)
 + * Add key validation to ssstablescrub (CASSANDRA-16969)
 + * Update Jackson from 2.9.10 to 2.12.5 (CASSANDRA-16851)
 + * Include SASI components to snapshots (CASSANDRA-15134)
 + * Make assassinate more resilient to missing tokens (CASSANDRA-16847)
 + * Exclude Jackson 1.x transitive dependency of hadoop* provided dependencies (CASSANDRA-16854)
 + * Validate SASI tokenizer options before adding index to schema (CASSANDRA-15135)
 + * Fixup scrub output when no data post-scrub and clear up old use of row, which really means partition (CASSANDRA-16835)
 + * Fix ant-junit dependency issue (CASSANDRA-16827)
 + * Reduce thread contention in CommitLogSegment and HintsBuffer (CASSANDRA-16072)
 + * Avoid sending CDC column if not enabled (CASSANDRA-16770)
 +Merged from 3.0:
+  * Fix failure handling in inter-node communication (CASSANDRA-16334)
   * Log more information when a node runs out of commitlog space (CASSANDRA-11323)
   * Don't take snapshots when truncating system tables (CASSANDRA-16839)
   * Make -Dtest.methods consistently optional in all Ant test targets (CASSANDRA-17014)
diff --cc src/java/org/apache/cassandra/net/MessageDeliveryTask.java
index c91e9da,26e780f..3de84d8
--- a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
+++ b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
@@@ -23,20 -25,21 +25,21 @@@ import java.util.EnumSet
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
+ import org.apache.cassandra.db.Mutation;
  import org.apache.cassandra.db.filter.TombstoneOverwhelmingException;
 +import org.apache.cassandra.exceptions.RequestFailureReason;
  import org.apache.cassandra.gms.Gossiper;
  import org.apache.cassandra.index.IndexNotAvailableException;
 +import org.apache.cassandra.io.util.DataOutputBuffer;
  
  public class MessageDeliveryTask implements Runnable
  {
      private static final Logger logger = LoggerFactory.getLogger(MessageDeliveryTask.class);
  
-     private final MessageIn message;
+     private final MessageIn<?> message;
      private final int id;
 -    private final long constructionTime;
 -    private final boolean isCrossNodeTimestamp;
  
-     public MessageDeliveryTask(MessageIn message, int id)
 -    public MessageDeliveryTask(MessageIn<?> message, int id, long timestamp, boolean isCrossNodeTimestamp)
++    public MessageDeliveryTask(MessageIn<?> message, int id)
      {
          assert message != null;
          this.message = message;
@@@ -89,23 -93,20 +92,33 @@@
      {
          if (message.doCallbackOnFailure())
          {
 -            MessageOut<?> response = new MessageOut<>(MessagingService.Verb.INTERNAL_RESPONSE)
 +            MessageOut response = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE)
                                                  .withParameter(MessagingService.FAILURE_RESPONSE_PARAM, MessagingService.ONE_BYTE);
  
 +            if (t instanceof TombstoneOverwhelmingException)
 +            {
 +                try (DataOutputBuffer out = new DataOutputBuffer())
 +                {
 +                    out.writeShort(RequestFailureReason.READ_TOO_MANY_TOMBSTONES.code);
 +                    response = response.withParameter(MessagingService.FAILURE_REASON_PARAM, out.getData());
 +                }
 +                catch (IOException ex)
 +                {
 +                    throw new RuntimeException(ex);
 +                }
 +            }
 +
-             MessagingService.instance().sendReply(response, id, message.from);
+             InetAddress from;
+             byte[] fromBytes = message.parameters.get(Mutation.FORWARD_FROM);
+             try
+             {
+                 from = fromBytes != null ? InetAddress.getByAddress(fromBytes) : message.from;
+             }
+             catch (UnknownHostException e)
+             {
+                 throw new RuntimeException(e);
+             }
+             MessagingService.instance().sendReply(response, id, from);
          }
      }
  
diff --cc src/java/org/apache/cassandra/service/StorageProxy.java
index dc1db09,89f93f6..763b1d7
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@@ -1522,8 -1445,8 +1522,8 @@@ public class StorageProxy implements St
                  catch (Exception ex)
                  {
                      if (!(ex instanceof WriteTimeoutException))
-                         logger.error("Failed to apply mutation locally : {}", ex);
+                         logger.error("Failed to apply mutation locally : ", ex);
 -                    handler.onFailure(FBUtilities.getBroadcastAddress());
 +                    handler.onFailure(FBUtilities.getBroadcastAddress(), RequestFailureReason.UNKNOWN);
                  }
              }
  

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org