Return-Path: Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: (qmail 78851 invoked from network); 25 Feb 2011 22:06:47 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 25 Feb 2011 22:06:47 -0000 Received: (qmail 20409 invoked by uid 500); 25 Feb 2011 22:06:47 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 20381 invoked by uid 500); 25 Feb 2011 22:06:46 -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 20373 invoked by uid 99); 25 Feb 2011 22:06:46 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 25 Feb 2011 22:06:46 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.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; Fri, 25 Feb 2011 22:06:45 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id D1084238890B; Fri, 25 Feb 2011 22:06:24 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1074714 - in /cassandra/trunk: interface/ interface/thrift/gen-java/org/apache/cassandra/thrift/ src/java/org/apache/cassandra/cql/ src/java/org/apache/cassandra/service/ src/java/org/apache/cassandra/thrift/ Date: Fri, 25 Feb 2011 22:06:24 -0000 To: commits@cassandra.apache.org From: eevans@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20110225220624.D1084238890B@eris.apache.org> Author: eevans Date: Fri Feb 25 22:06:24 2011 New Revision: 1074714 URL: http://svn.apache.org/viewvc?rev=1074714&view=rev Log: raise IRE for null keyspace Patch by eevans; reviewed by jbellis for CASSANDRA-2251 Modified: cassandra/trunk/interface/cassandra.thrift cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java cassandra/trunk/src/java/org/apache/cassandra/service/ClientState.java cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java Modified: cassandra/trunk/interface/cassandra.thrift URL: http://svn.apache.org/viewvc/cassandra/trunk/interface/cassandra.thrift?rev=1074714&r1=1074713&r2=1074714&view=diff ============================================================================== --- cassandra/trunk/interface/cassandra.thrift (original) +++ cassandra/trunk/interface/cassandra.thrift Fri Feb 25 22:06:24 2011 @@ -633,7 +633,8 @@ service Cassandra { list describe_splits(1:required string cfName, 2:required string start_token, 3:required string end_token, - 4:required i32 keys_per_split), + 4:required i32 keys_per_split) + throws (1:InvalidRequestException ire), /** adds a column family. returns the new schema id. */ string system_add_column_family(1:required CfDef cf_def) Modified: cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java URL: http://svn.apache.org/viewvc/cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java?rev=1074714&r1=1074713&r2=1074714&view=diff ============================================================================== --- cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java (original) +++ cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java Fri Feb 25 22:06:24 2011 @@ -143,6 +143,11 @@ public class Cassandra { * that all the values in column_path besides column_path.column_family are truly optional: you can remove the entire * row by just specifying the ColumnFamily, or you can remove a SuperColumn or a single Column by specifying those levels too. * + * Note that counters have limited support for deletes: if you remove + * a counter, you must wait to issue any following update until the + * delete has reached all the nodes and all of them have been fully + * compacted. + * * @param key * @param column_path * @param timestamp @@ -294,7 +299,7 @@ public class Cassandra { * @param end_token * @param keys_per_split */ - public List describe_splits(String cfName, String start_token, String end_token, int keys_per_split) throws TException; + public List describe_splits(String cfName, String start_token, String end_token, int keys_per_split) throws InvalidRequestException, TException; /** * adds a column family. returns the new schema id. @@ -1620,7 +1625,7 @@ public class Cassandra { throw new TApplicationException(TApplicationException.MISSING_RESULT, "describe_keyspace failed: unknown result"); } - public List describe_splits(String cfName, String start_token, String end_token, int keys_per_split) throws TException + public List describe_splits(String cfName, String start_token, String end_token, int keys_per_split) throws InvalidRequestException, TException { send_describe_splits(cfName, start_token, end_token, keys_per_split); return recv_describe_splits(); @@ -1639,7 +1644,7 @@ public class Cassandra { oprot_.getTransport().flush(); } - public List recv_describe_splits() throws TException + public List recv_describe_splits() throws InvalidRequestException, TException { TMessage msg = iprot_.readMessageBegin(); if (msg.type == TMessageType.EXCEPTION) { @@ -1656,6 +1661,9 @@ public class Cassandra { if (result.isSetSuccess()) { return result.success; } + if (result.ire != null) { + throw result.ire; + } throw new TApplicationException(TApplicationException.MISSING_RESULT, "describe_splits failed: unknown result"); } @@ -2929,7 +2937,7 @@ public class Cassandra { prot.writeMessageEnd(); } - public List getResult() throws TException { + public List getResult() throws InvalidRequestException, TException { if (getState() != State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -4298,7 +4306,19 @@ public class Cassandra { } iprot.readMessageEnd(); describe_splits_result result = new describe_splits_result(); - result.success = iface_.describe_splits(args.cfName, args.start_token, args.end_token, args.keys_per_split); + try { + result.success = iface_.describe_splits(args.cfName, args.start_token, args.end_token, args.keys_per_split); + } catch (InvalidRequestException ire) { + result.ire = ire; + } catch (Throwable th) { + LOGGER.error("Internal error processing describe_splits", th); + TApplicationException x = new TApplicationException(TApplicationException.INTERNAL_ERROR, "Internal error processing describe_splits"); + oprot.writeMessageBegin(new TMessage("describe_splits", TMessageType.EXCEPTION, seqid)); + x.write(oprot); + oprot.writeMessageEnd(); + oprot.getTransport().flush(); + return; + } oprot.writeMessageBegin(new TMessage("describe_splits", TMessageType.REPLY, seqid)); result.write(oprot); oprot.writeMessageEnd(); @@ -30156,12 +30176,15 @@ public class Cassandra { private static final TStruct STRUCT_DESC = new TStruct("describe_splits_result"); private static final TField SUCCESS_FIELD_DESC = new TField("success", TType.LIST, (short)0); + private static final TField IRE_FIELD_DESC = new TField("ire", TType.STRUCT, (short)1); public List success; + public InvalidRequestException ire; /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements TFieldIdEnum { - SUCCESS((short)0, "success"); + SUCCESS((short)0, "success"), + IRE((short)1, "ire"); private static final Map byName = new HashMap(); @@ -30178,6 +30201,8 @@ public class Cassandra { switch(fieldId) { case 0: // SUCCESS return SUCCESS; + case 1: // IRE + return IRE; default: return null; } @@ -30225,6 +30250,8 @@ public class Cassandra { tmpMap.put(_Fields.SUCCESS, new FieldMetaData("success", TFieldRequirementType.DEFAULT, new ListMetaData(TType.LIST, new FieldValueMetaData(TType.STRING)))); + tmpMap.put(_Fields.IRE, new FieldMetaData("ire", TFieldRequirementType.DEFAULT, + new FieldValueMetaData(TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); FieldMetaData.addStructMetaDataMap(describe_splits_result.class, metaDataMap); } @@ -30233,10 +30260,12 @@ public class Cassandra { } public describe_splits_result( - List success) + List success, + InvalidRequestException ire) { this(); this.success = success; + this.ire = ire; } /** @@ -30250,6 +30279,9 @@ public class Cassandra { } this.success = __this__success; } + if (other.isSetIre()) { + this.ire = new InvalidRequestException(other.ire); + } } public describe_splits_result deepCopy() { @@ -30259,6 +30291,7 @@ public class Cassandra { @Override public void clear() { this.success = null; + this.ire = null; } public int getSuccessSize() { @@ -30300,6 +30333,30 @@ public class Cassandra { } } + public InvalidRequestException getIre() { + return this.ire; + } + + public describe_splits_result setIre(InvalidRequestException ire) { + this.ire = ire; + return this; + } + + public void unsetIre() { + this.ire = null; + } + + /** Returns true if field ire is set (has been asigned a value) and false otherwise */ + public boolean isSetIre() { + return this.ire != null; + } + + public void setIreIsSet(boolean value) { + if (!value) { + this.ire = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: @@ -30310,6 +30367,14 @@ public class Cassandra { } break; + case IRE: + if (value == null) { + unsetIre(); + } else { + setIre((InvalidRequestException)value); + } + break; + } } @@ -30318,6 +30383,9 @@ public class Cassandra { case SUCCESS: return getSuccess(); + case IRE: + return getIre(); + } throw new IllegalStateException(); } @@ -30331,6 +30399,8 @@ public class Cassandra { switch (field) { case SUCCESS: return isSetSuccess(); + case IRE: + return isSetIre(); } throw new IllegalStateException(); } @@ -30357,6 +30427,15 @@ public class Cassandra { return false; } + boolean this_present_ire = true && this.isSetIre(); + boolean that_present_ire = true && that.isSetIre(); + if (this_present_ire || that_present_ire) { + if (!(this_present_ire && that_present_ire)) + return false; + if (!this.ire.equals(that.ire)) + return false; + } + return true; } @@ -30369,6 +30448,11 @@ public class Cassandra { if (present_success) builder.append(success); + boolean present_ire = true && (isSetIre()); + builder.append(present_ire); + if (present_ire) + builder.append(ire); + return builder.toHashCode(); } @@ -30390,6 +30474,16 @@ public class Cassandra { return lastComparison; } } + lastComparison = Boolean.valueOf(isSetIre()).compareTo(typedOther.isSetIre()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetIre()) { + lastComparison = TBaseHelper.compareTo(this.ire, typedOther.ire); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -30424,6 +30518,14 @@ public class Cassandra { TProtocolUtil.skip(iprot, field.type); } break; + case 1: // IRE + if (field.type == TType.STRUCT) { + this.ire = new InvalidRequestException(); + this.ire.read(iprot); + } else { + TProtocolUtil.skip(iprot, field.type); + } + break; default: TProtocolUtil.skip(iprot, field.type); } @@ -30449,6 +30551,10 @@ public class Cassandra { oprot.writeListEnd(); } oprot.writeFieldEnd(); + } else if (this.isSetIre()) { + oprot.writeFieldBegin(IRE_FIELD_DESC); + this.ire.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -30466,6 +30572,14 @@ public class Cassandra { sb.append(this.success); } first = false; + if (!first) sb.append(", "); + sb.append("ire:"); + if (this.ire == null) { + sb.append("null"); + } else { + sb.append(this.ire); + } + first = false; sb.append(")"); return sb.toString(); } 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=1074714&r1=1074713&r2=1074714&view=diff ============================================================================== --- cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java (original) +++ cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java Fri Feb 25 22:06:24 2011 @@ -382,7 +382,11 @@ public class QueryProcessor CqlParser parser = getParser(queryString); CQLStatement statement = parser.query(); parser.throwLastRecognitionError(); - String keyspace = clientState.getKeyspace(); + String keyspace = null; + + // Chicken-and-egg; No keyspace to get when we're setting one. + if (statement.type != StatementType.USE) + keyspace = clientState.getKeyspace(); CqlResult avroResult = new CqlResult(); Modified: cassandra/trunk/src/java/org/apache/cassandra/service/ClientState.java URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/service/ClientState.java?rev=1074714&r1=1074713&r2=1074714&view=diff ============================================================================== --- cassandra/trunk/src/java/org/apache/cassandra/service/ClientState.java (original) +++ cassandra/trunk/src/java/org/apache/cassandra/service/ClientState.java Fri Feb 25 22:06:24 2011 @@ -55,8 +55,10 @@ public class ClientState reset(); } - public String getKeyspace() + public String getKeyspace() throws InvalidRequestException { + if (keyspace == null) + throw new InvalidRequestException("no keyspace has been specified"); return keyspace; } Modified: cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java?rev=1074714&r1=1074713&r2=1074714&view=diff ============================================================================== --- cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java (original) +++ cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java Fri Feb 25 22:06:24 2011 @@ -640,7 +640,8 @@ public class CassandraServer implements return DatabaseDescriptor.getEndpointSnitch().getClass().getName(); } - public List describe_splits(String cfName, String start_token, String end_token, int keys_per_split) throws TException + public List describe_splits(String cfName, String start_token, String end_token, int keys_per_split) + throws TException, InvalidRequestException { // TODO: add keyspace authorization call post CASSANDRA-1425 Token.TokenFactory tf = StorageService.getPartitioner().getTokenFactory();