You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2013/11/18 17:34:25 UTC

[4/7] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0

Merge branch 'cassandra-1.2' into cassandra-2.0

Conflicts:
	NEWS.txt
	src/java/org/apache/cassandra/cql3/QueryProcessor.java
	src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
	src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
	src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
	src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
	src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
	src/java/org/apache/cassandra/thrift/CassandraServer.java


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

Branch: refs/heads/trunk
Commit: 25471bac3527c9fc54c815626f9266d5ea8508da
Parents: a7a7ede 0ffa5c2
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Mon Nov 18 17:30:15 2013 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Mon Nov 18 17:30:15 2013 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 ++
 .../cql3/MeasurableForPreparedCache.java        | 26 ++++++++++++++++++++
 .../apache/cassandra/cql3/QueryProcessor.java   |  9 +++++--
 .../cql3/statements/BatchStatement.java         | 12 ++++++++-
 .../cql3/statements/ModificationStatement.java  |  9 ++++++-
 .../cql3/statements/SelectStatement.java        |  8 +++++-
 .../cassandra/dht/Murmur3Partitioner.java       |  9 ++++++-
 .../cassandra/thrift/CassandraServer.java       |  2 ++
 8 files changed, 71 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/25471bac/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 57ad75d,d7395a6..7b2db56
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -48,42 -21,11 +48,44 @@@ Merged from 1.2
   * Fix AssertionError when doing set element deletion (CASSANDRA-6341)
   * Make CL code for the native protocol match the one in C* 2.0
     (CASSANDRA-6347)
+  * Disallow altering CQL3 table from thrift (CASSANDRA-6370)
+  * Fix size computation of prepared statement (CASSANDRA-6369)
  
  
 -1.2.11
 +2.0.2
 + * Update FailureDetector to use nanontime (CASSANDRA-4925)
 + * Fix FileCacheService regressions (CASSANDRA-6149)
 + * Never return WriteTimeout for CL.ANY (CASSANDRA-6032)
 + * Fix race conditions in bulk loader (CASSANDRA-6129)
 + * Add configurable metrics reporting (CASSANDRA-4430)
 + * drop queries exceeding a configurable number of tombstones (CASSANDRA-6117)
 + * Track and persist sstable read activity (CASSANDRA-5515)
 + * Fixes for speculative retry (CASSANDRA-5932, CASSANDRA-6194)
 + * Improve memory usage of metadata min/max column names (CASSANDRA-6077)
 + * Fix thrift validation refusing row markers on CQL3 tables (CASSANDRA-6081)
 + * Fix insertion of collections with CAS (CASSANDRA-6069)
 + * Correctly send metadata on SELECT COUNT (CASSANDRA-6080)
 + * Track clients' remote addresses in ClientState (CASSANDRA-6070)
 + * Create snapshot dir if it does not exist when migrating
 +   leveled manifest (CASSANDRA-6093)
 + * make sequential nodetool repair the default (CASSANDRA-5950)
 + * Add more hooks for compaction strategy implementations (CASSANDRA-6111)
 + * Fix potential NPE on composite 2ndary indexes (CASSANDRA-6098)
 + * Delete can potentially be skipped in batch (CASSANDRA-6115)
 + * Allow alter keyspace on system_traces (CASSANDRA-6016)
 + * Disallow empty column names in cql (CASSANDRA-6136)
 + * Use Java7 file-handling APIs and fix file moving on Windows (CASSANDRA-5383)
 + * Save compaction history to system keyspace (CASSANDRA-5078)
 + * Fix NPE if StorageService.getOperationMode() is executed before full startup (CASSANDRA-6166)
 + * CQL3: support pre-epoch longs for TimestampType (CASSANDRA-6212)
 + * Add reloadtriggers command to nodetool (CASSANDRA-4949)
 + * cqlsh: ignore empty 'value alias' in DESCRIBE (CASSANDRA-6139)
 + * Fix sstable loader (CASSANDRA-6205)
 + * Reject bootstrapping if the node already exists in gossip (CASSANDRA-5571)
 + * Fix NPE while loading paxos state (CASSANDRA-6211)
 + * cqlsh: add SHOW SESSION <tracing-session> command (CASSANDRA-6228)
 +Merged from 1.2:
 + * (Hadoop) Require CFRR batchSize to be at least 2 (CASSANDRA-6114)
   * Add a warning for small LCS sstable size (CASSANDRA-6191)
   * Add ability to list specific KS/CF combinations in nodetool cfstats (CASSANDRA-4191)
   * Mark CF clean if a mutation raced the drop and got it marked dirty (CASSANDRA-5946)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25471bac/src/java/org/apache/cassandra/cql3/MeasurableForPreparedCache.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/MeasurableForPreparedCache.java
index 0000000,0000000..6b3b4b5
new file mode 100644
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/MeasurableForPreparedCache.java
@@@ -1,0 -1,0 +1,26 @@@
++/*
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *   http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing,
++ * software distributed under the License is distributed on an
++ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
++ * KIND, either express or implied.  See the License for the
++ * specific language governing permissions and limitations
++ * under the License.
++ */
++package org.apache.cassandra.cql3;
++
++import org.github.jamm.MemoryMeter;
++
++public interface MeasurableForPreparedCache
++{
++    public long measureForPreparedCache(MemoryMeter meter);
++}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25471bac/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25471bac/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index 3d42da6,d211eb9..311a3c7
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@@ -20,31 -20,27 +20,33 @@@ package org.apache.cassandra.cql3.state
  import java.nio.ByteBuffer;
  import java.util.*;
  
+ import org.github.jamm.MemoryMeter;
+ 
 -import org.apache.cassandra.auth.Permission;
  import org.apache.cassandra.cql3.*;
  import org.apache.cassandra.db.ConsistencyLevel;
 -import org.apache.cassandra.db.CounterMutation;
  import org.apache.cassandra.db.IMutation;
 -import org.apache.cassandra.db.RowMutation;
  import org.apache.cassandra.exceptions.*;
  import org.apache.cassandra.service.ClientState;
 +import org.apache.cassandra.service.QueryState;
 +import org.apache.cassandra.service.StorageProxy;
 +import org.apache.cassandra.transport.messages.ResultMessage;
  import org.apache.cassandra.utils.Pair;
 -import org.apache.cassandra.utils.ObjectSizes;
  
  /**
   * A <code>BATCH</code> statement parsed from a CQL query.
   *
   */
- public class BatchStatement implements CQLStatement
 -public class BatchStatement extends ModificationStatement
++public class BatchStatement implements CQLStatement, MeasurableForPreparedCache
  {
 -    // statements to execute
 -    protected final List<ModificationStatement> statements;
 +    public static enum Type
 +    {
 +        LOGGED, UNLOGGED, COUNTER
 +    }
 +
 +    private final int boundTerms;
 +    public final Type type;
 +    private final List<ModificationStatement> statements;
 +    private final Attributes attrs;
  
      /**
       * Creates a new BatchStatement from a list of statements and a
@@@ -54,19 -50,29 +56,27 @@@
       * @param statements a list of UpdateStatements
       * @param attrs additional attributes for statement (CL, timestamp, timeToLive)
       */
 -    public BatchStatement(Type type, List<ModificationStatement> statements, Attributes attrs)
 +    public BatchStatement(int boundTerms, Type type, List<ModificationStatement> statements, Attributes attrs)
      {
 -        super(null, attrs);
 +        this.boundTerms = boundTerms;
          this.type = type;
          this.statements = statements;
 +        this.attrs = attrs;
      }
  
+     public long measureForPreparedCache(MemoryMeter meter)
+     {
 -        long size = meter.measure(this) + meter.measure(statements);
++        long size = meter.measure(this) + meter.measure(statements) + meter.measureDeep(attrs);
+         for (ModificationStatement stmt : statements)
+             size += stmt.measureForPreparedCache(meter);
+         return size;
+     }
+ 
 -    @Override
 -    public void prepareKeyspace(ClientState state) throws InvalidRequestException
 +    public int getBoundsTerms()
      {
 -        for (ModificationStatement statement : statements)
 -            statement.prepareKeyspace(state);
 +        return boundTerms;
      }
  
 -    @Override
      public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException
      {
          for (ModificationStatement statement : statements)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25471bac/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index f5cff1d,bfbf511..8833f34
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@@ -20,8 -20,7 +20,10 @@@ package org.apache.cassandra.cql3.state
  import java.nio.ByteBuffer;
  import java.util.*;
  
++import org.github.jamm.MemoryMeter;
++
  import org.apache.cassandra.auth.Permission;
 +import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.cql3.*;
  import org.apache.cassandra.db.*;
  import org.apache.cassandra.db.filter.ColumnSlice;
@@@ -34,70 -31,35 +36,75 @@@ import org.apache.cassandra.exceptions.
  import org.apache.cassandra.service.ClientState;
  import org.apache.cassandra.service.QueryState;
  import org.apache.cassandra.service.StorageProxy;
 +import org.apache.cassandra.thrift.ThriftValidation;
  import org.apache.cassandra.transport.messages.ResultMessage;
 +import org.apache.cassandra.utils.Pair;
 +import org.apache.cassandra.utils.ByteBufferUtil;
  
 -/**
 - * Abstract class for statements that apply on a given column family.
 +/*
 + * Abstract parent class of individual modifications, i.e. INSERT, UPDATE and DELETE.
   */
- public abstract class ModificationStatement implements CQLStatement
 -public abstract class ModificationStatement extends CFStatement implements CQLStatement, MeasurableForPreparedCache
++public abstract class ModificationStatement implements CQLStatement, MeasurableForPreparedCache
  {
 -    public static enum Type
 +    private static final ColumnIdentifier CAS_RESULT_COLUMN = new ColumnIdentifier("[applied]", false);
 +
 +    public final CFMetaData cfm;
 +    public final Attributes attrs;
 +
 +    private final Map<ColumnIdentifier, Restriction> processedKeys = new HashMap<ColumnIdentifier, Restriction>();
 +    private final List<Operation> columnOperations = new ArrayList<Operation>();
 +
 +    private int boundTerms;
 +    private List<Operation> columnConditions;
 +    private boolean ifNotExists;
 +
 +    public ModificationStatement(CFMetaData cfm, Attributes attrs)
      {
 -        LOGGED, UNLOGGED, COUNTER
 +        this.cfm = cfm;
 +        this.attrs = attrs;
      }
  
 -    protected Type type;
++    public long measureForPreparedCache(MemoryMeter meter)
++    {
++        return meter.measureDeep(this) - meter.measureDeep(cfm);
++    }
++
 +    public abstract boolean requireFullClusteringKey();
 +    public abstract ColumnFamily updateForKey(ByteBuffer key, ColumnNameBuilder builder, UpdateParameters params) throws InvalidRequestException;
  
 -    private Long timestamp;
 -    private final int timeToLive;
 +    public int getBoundsTerms()
 +    {
 +        return boundTerms;
 +    }
  
 -    public ModificationStatement(CFName name, Attributes attrs)
 +    public String keyspace()
      {
 -        this(name, attrs.timestamp, attrs.timeToLive);
 +        return cfm.ksName;
      }
  
 -    public ModificationStatement(CFName name, Long timestamp, int timeToLive)
 +    public String columnFamily()
      {
 -        super(name);
 -        this.timestamp = timestamp;
 -        this.timeToLive = timeToLive;
 +        return cfm.cfName;
 +    }
 +
 +    public boolean isCounter()
 +    {
 +        return cfm.getDefaultValidator().isCommutative();
 +    }
 +
 +    public long getTimestamp(long now, List<ByteBuffer> variables) throws InvalidRequestException
 +    {
 +        return attrs.getTimestamp(now, variables);
 +    }
 +
 +    public boolean isTimestampSet()
 +    {
 +        return attrs.isTimestampSet();
 +    }
 +
 +    public int getTimeToLive(List<ByteBuffer> variables) throws InvalidRequestException
 +    {
 +        return attrs.getTimeToLive(variables);
      }
  
      public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25471bac/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 4b9a334,e1537fc..b94e549
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -19,11 -19,12 +19,12 @@@ package org.apache.cassandra.cql3.state
  
  import java.nio.ByteBuffer;
  import java.util.*;
 -import java.util.concurrent.ExecutionException;
  
 +import com.google.common.base.Objects;
 +import com.google.common.base.Predicate;
  import com.google.common.collect.AbstractIterator;
 -import org.slf4j.Logger;
 -import org.slf4j.LoggerFactory;
 +import com.google.common.collect.Iterables;
+ import org.github.jamm.MemoryMeter;
  
  import org.apache.cassandra.auth.Permission;
  import org.apache.cassandra.cql3.*;
@@@ -54,9 -54,9 +55,9 @@@ import org.apache.cassandra.utils.Pair
   * column family, expression, result count, and ordering clause.
   *
   */
- public class SelectStatement implements CQLStatement
+ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
  {
 -    private static final Logger logger = LoggerFactory.getLogger(SelectStatement.class);
 +    private static final int DEFAULT_COUNT_PAGE_SIZE = 10000;
  
      private final int boundTerms;
      public final CFDefinition cfDef;
@@@ -91,24 -98,13 +92,29 @@@
          this.keyRestrictions = new Restriction[cfDef.keys.size()];
          this.columnRestrictions = new Restriction[cfDef.columns.size()];
          this.parameters = parameters;
 +        this.limit = limit;
 +    }
 +
 +    // Creates a simple select based on the given selection.
 +    // Note that the results select statement should not be used for actual queries, but only for processing already
 +    // queried data through processColumnFamily.
 +    static SelectStatement forSelection(CFDefinition cfDef, Selection selection)
 +    {
 +        return new SelectStatement(cfDef, 0, defaultParameters, selection, null);
 +    }
 +
 +    public ResultSet.Metadata getResultMetadata()
 +    {
 +        return parameters.isCount
 +             ? ResultSet.makeCountMetadata(keyspace(), columnFamily(), parameters.countAlias)
 +             : selection.getResultMetadata();
      }
  
+     public long measureForPreparedCache(MemoryMeter meter)
+     {
+         return meter.measureDeep(this) - meter.measureDeep(cfDef);
+     }
+ 
      public int getBoundsTerms()
      {
          return boundTerms;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25471bac/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25471bac/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/CassandraServer.java
index d52eb76,5b9fbfd..d710352
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@@ -1629,16 -1407,16 +1629,18 @@@ public class CassandraServer implement
              if (oldCfm == null)
                  throw new InvalidRequestException("Could not find column family definition to modify.");
  
 -            if (!oldCfm.isThriftIncompatible())
++            if (!oldCfm.isThriftCompatible())
+                 throw new InvalidRequestException("Cannot modify CQL3 table " + oldCfm.cfName + " as it may break the schema. You should use cqlsh to modify CQL3 tables instead.");
  
 -            state().hasColumnFamilyAccess(cf_def.keyspace, cf_def.name, Permission.ALTER);
 -
              CFMetaData.applyImplicitDefaults(cf_def);
              CFMetaData cfm = CFMetaData.fromThrift(cf_def);
 -            CFMetaData.validateCompactionOptions(cfm.compactionStrategyClass, cfm.compactionStrategyOptions, false);
 +            CFMetaData.validateCompactionOptions(cfm.compactionStrategyClass, cfm.compactionStrategyOptions);
              cfm.addDefaultIndexNames();
 -            MigrationManager.announceColumnFamilyUpdate(cfm);
 +
 +            if (!oldCfm.getTriggers().equals(cfm.getTriggers()))
 +                state().ensureIsSuper("Only superusers are allowed to add or remove triggers.");
 +
 +            MigrationManager.announceColumnFamilyUpdate(cfm, true);
              return Schema.instance.getVersion().toString();
          }
          catch (RequestValidationException e)