Return-Path: Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: (qmail 3497 invoked from network); 13 Dec 2010 19:50:31 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 13 Dec 2010 19:50:31 -0000 Received: (qmail 20570 invoked by uid 500); 13 Dec 2010 19:50:31 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 20537 invoked by uid 500); 13 Dec 2010 19:50:31 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 20529 invoked by uid 99); 13 Dec 2010 19:50:31 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 13 Dec 2010 19:50:31 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 13 Dec 2010 19:50:30 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id 603E823889B2; Mon, 13 Dec 2010 19:50:10 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1045342 - in /cassandra/trunk: doc/cql/ src/java/org/apache/cassandra/cql/ test/system/ Date: Mon, 13 Dec 2010 19:50:10 -0000 To: commits@cassandra.apache.org From: eevans@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20101213195010.603E823889B2@eris.apache.org> Author: eevans Date: Mon Dec 13 19:50:09 2010 New Revision: 1045342 URL: http://svn.apache.org/viewvc?rev=1045342&view=rev Log: CASSANDRA-1706: CQL DELETE w/ functional tests Patch by eevans for CASSANDRA-1706 Added: cassandra/trunk/src/java/org/apache/cassandra/cql/DeleteStatement.java Modified: cassandra/trunk/doc/cql/CQL.html cassandra/trunk/doc/cql/CQL.textile cassandra/trunk/src/java/org/apache/cassandra/cql/Cql.g cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java cassandra/trunk/src/java/org/apache/cassandra/cql/StatementType.java cassandra/trunk/test/system/test_cql.py Modified: cassandra/trunk/doc/cql/CQL.html URL: http://svn.apache.org/viewvc/cassandra/trunk/doc/cql/CQL.html?rev=1045342&r1=1045341&r2=1045342&view=diff ============================================================================== --- cassandra/trunk/doc/cql/CQL.html (original) +++ cassandra/trunk/doc/cql/CQL.html Mon Dec 13 19:50:09 2010 @@ -1,4 +1,4 @@ -

Cassandra Query Language (CQL) v0.99.1

Table of Contents

  1. Cassandra Query Language (CQL) v0.99.1
    1. Table of Contents
    2. USE
    3. SELECT
      1. Specifying Columns
      2. Column Family
      3. Consistency Level
      4. Filtering rows
      5. Limits
    4. UPDATE
      1. Column Family
      2. Consistency Level
      3. Specifying Columns and Row
    5. TRUNCATE
    6. Common Idioms
      1. Specifying Consistency
      2. Term specification
        1. String Literals
        2. Integers / longs

USE

Synopsis:

USE <KEYSPACE>;
+

Cassandra Query Language (CQL) v0.99.1

Table of Contents

  1. Cassandra Query Language (CQL) v0.99.1
    1. Table of Contents
    2. USE
    3. SELECT
      1. Specifying Columns
      2. Column Family
      3. Consistency Level
      4. Filtering rows
      5. Limits
    4. UPDATE
      1. Column Family
      2. Consistency Level
      3. Specifying Columns and Row
    5. DELETE
      1. Specifying Columns
      2. Column Family
      3. Consistency Level
      4. Specifying Rows
    6. TRUNCATE
    7. Common Idioms
      1. Specifying Consistency
      2. Term specification
        1. String Literals
        2. Integers / longs

USE

Synopsis:

<
 code>USE <KEYSPACE>;
 

A USE statement consists of the USE keyword, followed by a valid keyspace name. Its purpose is to assign the per-connection, current working keyspace. All subsequent keyspace-specific actions will be performed in the context of the supplied value.

SELECT

Synopsis:

SELECT [FIRST N] [REVERSED] <SELECT EXPR> FROM <COLUMN FAMILY> [USING <CONSISTENCY>]
         [WHERE <CLAUSE>] [LIMIT N];
 

A SELECT is used to read one or more records from a Cassandra column family. It returns a result-set of rows, where each row consists of a key and a collection of columns corresponding to the query.

Specifying Columns

SELECT [FIRST N] [REVERSED] name1, name2, name3 FROM ...
@@ -18,6 +18,13 @@ UPDATE CF1 SET name1 = value1, name2 = v
 UPDATE CF1 SET name3 = value3 WHERE KEY = keyname2;
 UPDATE CF2 SET name4 = value4, name5 = value5 WHERE KEY = keyname3;
 APPLY BATCH
-

When batching UPDATEs, a single consistency level is used for the entire batch, it appears after the BEGIN BATCH statement, and uses the standard consistency level specification. Batch UPDATEs default to CONSISTENCY.ONE when left unspecified.

NOTE: While there are no isolation guarantees, UPDATE queries are atomic within a give record.

TRUNCATE

Synopsis:

TRUNCATE <COLUMN FAMILY>
+

When batching UPDATEs, a single consistency level is used for the entire batch, it appears after the BEGIN BATCH statement, and uses the standard consistency level specification. Batch UPDATEs default to CONSISTENCY.ONE when left unspecified.

NOTE: While there are no isolation guarantees, UPDATE queries are atomic within a give record.

DELETE

Synopsis:

DELETE [COLUMNS] FROM <COLUMN FAMILY> [USING <CONSISTENCY>] WHERE KEY = keyname1
+DELETE [COLUMNS] FROM <COLUMN FAMILY> [USING <CONSISTENCY>] WHERE KEY IN (keyname1, keyname2);
+

A DELETE is used to perform the removal of one or more columns from one or more rows.

Specifying Columns

DELETE [COLUMNS] ...
+

Following the DELETE keyword is an optional comma-delimited list of column name terms. When no column names are specified, the remove applies to the entire row(s) matched by the WHERE clause

Column Family

DELETE ... FROM <COLUMN FAMILY> ...
+

The column family name follows the list of column names.

Consistency Level

UPDATE ... [USING <CONSISTENCY>] ...
+

Following the column family identifier is an optional consistency level specification.

Specifying Rows

UPDATE ... WHERE KEY = keyname1
+UPDATE ... WHERE KEY IN (keyname1, keyname2)
+

The WHERE clause is used to determine which row(s) a DELETE applies to. The first form allows the specification of a single keyname using the KEY keyword and the = operator. The second form allows a list of keyname terms to be specified using the IN notation and a parenthesized list of comma-delimited keyname terms.

TRUNCATE

Synopsis:

TRUNCATE <COLUMN FAMILY>
 

Accepts a single argument for the column family name, and permanently removes all data from said column family.

Common Idioms

Specifying Consistency

... USING <CONSISTENCY> ...
 

Consistency level specifications are made up the keyword USING, followed by a consistency level identifier. Valid consistency levels are as follows:

  • CONSISTENCY.ZERO
  • CONSISTENCY.ONE (default)
  • CONSISTENCY.QUORUM
  • CONSISTENCY.ALL
  • CONSISTENCY.DCQUORUM
  • CONSISTENCY.DCQUORUMSYNC

Term specification

Where possible, the type of terms are inferred; the following term types are supported:

String Literals

String literals are any value enclosed in double-quotes, (`"`). String literals are treated as raw bytes; no interpolation is performed.

Integers / longs

Integers are any term consisting soley of unquoted numericals, longs are any otherwise valid integer term followed by an upper case “L”, (e.g. 100L). It is an error to s pecify an integer term that will not fit in 4 bytes unsigned, or a long that will not fit in 8 bytes unsigned.

\ No newline at end of file Modified: cassandra/trunk/doc/cql/CQL.textile URL: http://svn.apache.org/viewvc/cassandra/trunk/doc/cql/CQL.textile?rev=1045342&r1=1045341&r2=1045342&view=diff ============================================================================== --- cassandra/trunk/doc/cql/CQL.textile (original) +++ cassandra/trunk/doc/cql/CQL.textile Mon Dec 13 19:50:09 2010 @@ -112,6 +112,45 @@ When batching UPDATEs, a single consiste _NOTE: While there are no isolation guarantees, @UPDATE@ queries are atomic within a give record._ +h2. DELETE + +_Synopsis:_ + +bc. +DELETE [COLUMNS] FROM [USING ] WHERE KEY = keyname1 +DELETE [COLUMNS] FROM [USING ] WHERE KEY IN (keyname1, keyname2); + +A @DELETE@ is used to perform the removal of one or more columns from one or more rows. + +h3. Specifying Columns + +bc. +DELETE [COLUMNS] ... + +Following the @DELETE@ keyword is an optional comma-delimited list of column name terms. When no column names are specified, the remove applies to the entire row(s) matched by the "@WHERE@ clause":#deleterows + +h3. Column Family + +bc. +DELETE ... FROM ... + +The column family name follows the list of column names. + +h3. Consistency Level + +bc. +UPDATE ... [USING ] ... + +Following the column family identifier is an optional "consistency level specification":#consistency. + +h3(#deleterows). Specifying Rows + +bc. +UPDATE ... WHERE KEY = keyname1 +UPDATE ... WHERE KEY IN (keyname1, keyname2) + +The @WHERE@ clause is used to determine which row(s) a @DELETE@ applies to. The first form allows the specification of a single keyname using the @KEY@ keyword and the @=@ operator. The second form allows a list of keyname terms to be specified using the @IN@ notation and a parenthesized list of comma-delimited keyname terms. + h2. TRUNCATE _Synopsis:_ Modified: cassandra/trunk/src/java/org/apache/cassandra/cql/Cql.g URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/cql/Cql.g?rev=1045342&r1=1045341&r2=1045342&view=diff ============================================================================== --- cassandra/trunk/src/java/org/apache/cassandra/cql/Cql.g (original) +++ cassandra/trunk/src/java/org/apache/cassandra/cql/Cql.g Mon Dec 13 19:50:09 2010 @@ -8,6 +8,7 @@ options { package org.apache.cassandra.cql; import java.util.Map; import java.util.HashMap; + import java.util.Collections; import org.apache.cassandra.thrift.ConsistencyLevel; import org.apache.cassandra.avro.InvalidRequestException; } @@ -48,6 +49,7 @@ query returns [CQLStatement stmnt] | batchUpdateStatement { $stmnt = new CQLStatement(StatementType.BATCH_UPDATE, $batchUpdateStatement.expr); } | useStatement { $stmnt = new CQLStatement(StatementType.USE, $useStatement.keyspace); } | truncateStatement { $stmnt = new CQLStatement(StatementType.TRUNCATE, $truncateStatement.cfam); } + | deleteStatement { $stmnt = new CQLStatement(StatementType.DELETE, $deleteStatement.expr); } ; // USE ; @@ -136,12 +138,44 @@ updateStatement returns [UpdateStatement } ; +/** + * DELETE + * name1, name2 + * FROM + * + * USING + * CONSISTENCY. + * WHERE + * KEY = keyname; + */ +deleteStatement returns [DeleteStatement expr] + : { + ConsistencyLevel cLevel = ConsistencyLevel.ONE; + List keyList = null; + List columnsList = Collections.emptyList(); + } + K_DELETE + ( cols=termList { columnsList = $cols.items; })? + K_FROM columnFamily=IDENT ( K_USING K_CONSISTENCY '.' K_LEVEL )? + K_WHERE ( K_KEY '=' key=term { keyList = Collections.singletonList(key); } + | K_KEY K_IN '(' keys=termList { keyList = $keys.items; } ')' + )? + { + return new DeleteStatement(columnsList, $columnFamily.text, cLevel, keyList); + } + ; + // TODO: date/time, utf8 term returns [Term item] : ( t=STRING_LITERAL | t=LONG ) { $item = new Term($t.text, $t.type); } ; +termList returns [List items] + : { $items = new ArrayList(); } + t1=term { $items.add(t1); } (',' tN=term { $items.add(tN); })* + ; + // Note: ranges are inclusive so >= and >, and < and <= all have the same semantics. relation returns [Relation rel] : { Term entity = new Term("KEY", STRING_LITERAL); } @@ -208,6 +242,8 @@ K_BEGIN: B E G I N; K_APPLY: A P P L Y; K_BATCH: B A T C H; K_TRUNCATE: T R U N C A T E; +K_DELETE: D E L E T E; +K_IN: I N; // Case-insensitive alpha characters fragment A: ('a'|'A'); Added: cassandra/trunk/src/java/org/apache/cassandra/cql/DeleteStatement.java URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/cql/DeleteStatement.java?rev=1045342&view=auto ============================================================================== --- cassandra/trunk/src/java/org/apache/cassandra/cql/DeleteStatement.java (added) +++ cassandra/trunk/src/java/org/apache/cassandra/cql/DeleteStatement.java Mon Dec 13 19:50:09 2010 @@ -0,0 +1,74 @@ +/* + * + * 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.cql; + +import java.util.List; + +import org.apache.cassandra.thrift.ConsistencyLevel; + +/** + * A DELETE parsed from a CQL query statement. + * + */ +public class DeleteStatement +{ + private List columns; + private String columnFamily; + private ConsistencyLevel cLevel; + private List keys; + + public DeleteStatement(List columns, String columnFamily, ConsistencyLevel cLevel, List keys) + { + this.columns = columns; + this.columnFamily = columnFamily; + this.cLevel = cLevel; + this.keys = keys; + } + + public List getColumns() + { + return columns; + } + + public String getColumnFamily() + { + return columnFamily; + } + + public ConsistencyLevel getConsistencyLevel() + { + return cLevel; + } + + public List getKeys() + { + return keys; + } + + public String toString() + { + return String.format("DeleteStatement(columns=%s, columnFamily=%s, consistency=%s keys=%s)", + columns, + columnFamily, + cLevel, + keys); + } +} Modified: cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java?rev=1045342&r1=1045341&r2=1045342&view=diff ============================================================================== --- cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java (original) +++ cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java Mon Dec 13 19:50:09 2010 @@ -378,6 +378,8 @@ public class QueryProcessor for (IColumn column : row.cf.getSortedColumns()) { + if (column.isMarkedForDelete()) + continue; Column avroColumn = new Column(); avroColumn.name = column.name(); avroColumn.value = column.value(); @@ -443,6 +445,40 @@ public class QueryProcessor avroResult.type = CqlResultType.VOID; return avroResult; + + case DELETE: + DeleteStatement delete = (DeleteStatement)statement.statement; + + List rowMutations = new ArrayList(); + for (Term key : delete.getKeys()) + { + RowMutation rm = new RowMutation(keyspace, key.getByteBuffer()); + if (delete.getColumns().size() < 1) // No columns, delete the row + rm.delete(new QueryPath(delete.getColumnFamily()), System.currentTimeMillis()); + else // Delete specific columns + { + for (Term column : delete.getColumns()) + rm.delete(new QueryPath(delete.getColumnFamily(), null, column.getByteBuffer()), + System.currentTimeMillis()); + } + rowMutations.add(rm); + } + + try + { + StorageProxy.mutate(rowMutations, delete.getConsistencyLevel()); + } + catch (org.apache.cassandra.thrift.UnavailableException e) + { + throw newUnavailableException(e); + } + catch (TimeoutException e) + { + throw new TimedOutException(); + } + + avroResult.type = CqlResultType.VOID; + return avroResult; } return null; // We should never get here. Modified: cassandra/trunk/src/java/org/apache/cassandra/cql/StatementType.java URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/cql/StatementType.java?rev=1045342&r1=1045341&r2=1045342&view=diff ============================================================================== --- cassandra/trunk/src/java/org/apache/cassandra/cql/StatementType.java (original) +++ cassandra/trunk/src/java/org/apache/cassandra/cql/StatementType.java Mon Dec 13 19:50:09 2010 @@ -22,5 +22,5 @@ package org.apache.cassandra.cql; public enum StatementType { - SELECT, UPDATE, BATCH_UPDATE, USE, TRUNCATE; + SELECT, UPDATE, BATCH_UPDATE, USE, TRUNCATE, DELETE; } Modified: cassandra/trunk/test/system/test_cql.py URL: http://svn.apache.org/viewvc/cassandra/trunk/test/system/test_cql.py?rev=1045342&r1=1045341&r2=1045342&view=diff ============================================================================== --- cassandra/trunk/test/system/test_cql.py (original) +++ cassandra/trunk/test/system/test_cql.py Mon Dec 13 19:50:09 2010 @@ -166,3 +166,38 @@ class TestCql(AvroTester): conn.execute('TRUNCATE Standard1;') r = conn.execute('SELECT "cd1" FROM Standard1 WHERE KEY = "kd"') assert len(r) == 0 + + def test_delete_columns(self): + "delete columns from a row" + conn = init() + r = conn.execute('SELECT "cd1", "col" FROM Standard1 WHERE KEY = "kd"') + assert "cd1" in [i['name'] for i in r[0]['columns']] + assert "col" in [i['name'] for i in r[0]['columns']] + conn.execute('DELETE "cd1", "col" FROM Standard1 WHERE KEY = "kd"') + r = conn.execute('SELECT "cd1", "col" FROM Standard1 WHERE KEY = "kd"') + assert len(r[0]['columns']) == 0 + + def test_delete_columns_multi_rows(self): + "delete columns from multiple rows" + conn = init() + r = conn.execute('SELECT "col" FROM Standard1 WHERE KEY = "kc"') + assert len(r[0]['columns']) == 1 + r = conn.execute('SELECT "col" FROM Standard1 WHERE KEY = "kd"') + assert len(r[0]['columns']) == 1 + + conn.execute('DELETE "col" FROM Standard1 WHERE KEY IN ("kc", "kd")') + r = conn.execute('SELECT "col" FROM Standard1 WHERE KEY = "kc"') + assert len(r[0]['columns']) == 0 + r = conn.execute('SELECT "col" FROM Standard1 WHERE KEY = "kd"') + assert len(r[0]['columns']) == 0 + + def test_delete_rows(self): + "delete entire rows" + conn = init() + r = conn.execute('SELECT "cd1", "col" FROM Standard1 WHERE KEY = "kd"') + assert "cd1" in [i['name'] for i in r[0]['columns']] + assert "col" in [i['name'] for i in r[0]['columns']] + conn.execute('DELETE FROM Standard1 WHERE KEY = "kd"') + r = conn.execute('SELECT "cd1", "col" FROM Standard1 WHERE KEY = "kd"') + assert len(r[0]['columns']) == 0 +