You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by pm...@apache.org on 2009/03/02 07:12:49 UTC

svn commit: r749205 [7/16] - in /incubator/cassandra/src/org/apache/cassandra: analytics/ cli/ concurrent/ config/ continuations/ cql/ cql/common/ cql/compiler/ cql/compiler/common/ cql/compiler/parse/ cql/compiler/sem/ cql/driver/ cql/execution/ dht/ ...

Added: incubator/cassandra/src/org/apache/cassandra/cql/common/CqlResult.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/common/CqlResult.java?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/common/CqlResult.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/common/CqlResult.java Mon Mar  2 06:12:46 2009
@@ -0,0 +1,36 @@
+/**
+ * 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.common;
+
+import java.util.List;
+import java.util.Map;
+
+public class CqlResult
+{
+    public int                       errorCode; // 0 - success
+    public String                    errorTxt;
+    public List<Map<String, String>> resultSet;
+
+    public CqlResult(List<Map<String, String>> rows)
+    {
+        resultSet = rows;
+        errorTxt  = null;
+        errorCode = 0; // success
+    }
+    
+};

Added: incubator/cassandra/src/org/apache/cassandra/cql/common/DMLPlan.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/common/DMLPlan.java?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/common/DMLPlan.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/common/DMLPlan.java Mon Mar  2 06:12:46 2009
@@ -0,0 +1,25 @@
+/**
+ * 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.common;
+
+/**
+ * This class represents the execution plan for DML (data manipulation language)
+ * CQL statements. 
+ */
+public abstract class DMLPlan extends Plan {};
\ No newline at end of file

Added: incubator/cassandra/src/org/apache/cassandra/cql/common/ExplainPlan.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/common/ExplainPlan.java?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/common/ExplainPlan.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/common/ExplainPlan.java Mon Mar  2 06:12:46 2009
@@ -0,0 +1,66 @@
+/**
+ * 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.common;
+
+import java.util.*;
+import org.apache.log4j.Logger;
+
+/**
+ * The "Plan" for the EXPLAIN PLAN statement itself!
+ * 
+ * It is nothing but a simple wrapper around the "Plan" for the statement
+ * on which an EXPLAIN PLAN has been requested.
+ */
+public class ExplainPlan extends Plan
+{
+    private final static Logger logger_ = Logger.getLogger(ExplainPlan.class);
+    
+    // the execution plan for the statement on which an 
+    // EXPLAIN PLAN was requested.
+    private Plan plan_ = null;
+
+    /**
+     *  Construct an ExplainPlan instance for the statement whose
+     * "plan" has been passed in.
+     */
+    public ExplainPlan(Plan plan)
+    {
+        plan_ = plan;
+    }
+
+    public CqlResult execute()
+    {
+        String planText = plan_.explainPlan();
+
+        List<Map<String, String>> rows = new LinkedList<Map<String, String>>(); 
+        Map<String, String> row = new HashMap<String, String>();
+        row.put("PLAN", planText);
+        rows.add(row);
+        
+        return new CqlResult(rows);
+    }
+
+    public String explainPlan()
+    {
+        // We never expect this method to get invoked for ExplainPlan instances
+        // (i.e. those that correspond to the EXPLAIN PLAN statement).
+        logger_.error("explainPlan() invoked on an ExplainPlan instance");
+        return null;
+    }
+}
\ No newline at end of file

Added: incubator/cassandra/src/org/apache/cassandra/cql/common/OperandDef.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/common/OperandDef.java?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/common/OperandDef.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/common/OperandDef.java Mon Mar  2 06:12:46 2009
@@ -0,0 +1,33 @@
+/**
+ * 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.common;
+
+/**
+ * OperandDef:
+ *
+ * The abstract definition of an operand (i.e. data item) in 
+ * CQL compiler/runtime. Examples, include a Constant operand
+ * or a Bind operand. This is the part of an operand definition
+ * that lives in the share-able execution plan.
+ */
+public abstract interface OperandDef
+{
+    public abstract Object get();
+    public abstract String explain();
+};
\ No newline at end of file

Added: incubator/cassandra/src/org/apache/cassandra/cql/common/Pair.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/common/Pair.java?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/common/Pair.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/common/Pair.java Mon Mar  2 06:12:46 2009
@@ -0,0 +1,53 @@
+/**
+ * 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.common;
+
+/* Would have expected java.util.* to have this class!
+ * Code cut-paste from wikipedia.
+ */
+
+/**
+ * Generic for representing a "typed" 2-tuple.
+ */
+public class Pair<T, S>
+{
+  public Pair(T f, S s)
+  { 
+    first = f;
+    second = s;   
+  }
+
+  public T getFirst()
+  {
+    return first;
+  }
+ 
+  public S getSecond() 
+  {
+    return second;
+  }
+ 
+  public String toString()
+  { 
+    return "(" + first.toString() + ", " + second.toString() + ")"; 
+  }
+ 
+  private T first;
+  private S second;
+}

Added: incubator/cassandra/src/org/apache/cassandra/cql/common/Plan.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/common/Plan.java?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/common/Plan.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/common/Plan.java Mon Mar  2 06:12:46 2009
@@ -0,0 +1,30 @@
+/**
+ * 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.common;
+
+/**
+ * Abstract class representing the shared execution plan for a CQL
+ * statement (query or DML operation).
+ * 
+ */
+public abstract class Plan
+{
+    public abstract CqlResult execute();
+    public abstract String explainPlan();
+}

Added: incubator/cassandra/src/org/apache/cassandra/cql/common/QueryPlan.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/common/QueryPlan.java?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/common/QueryPlan.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/common/QueryPlan.java Mon Mar  2 06:12:46 2009
@@ -0,0 +1,53 @@
+/**
+ * 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.common;
+
+import org.apache.log4j.Logger;
+
+/**
+ * This class represents the execution plan for Query (data retrieval) statement. 
+ */
+public class QueryPlan extends Plan
+{
+    private final static Logger logger_ = Logger.getLogger(QueryPlan.class);    
+
+    public RowSourceDef root;    // the root of the row source tree
+
+    public QueryPlan(RowSourceDef rwsDef)
+    {
+        root = rwsDef;
+    }
+    
+    public CqlResult execute()
+    {
+        if (root != null)
+        {
+            return new CqlResult(root.getRows());
+        }
+        else
+            logger_.error("No rowsource to execute");
+        return null;
+    }
+    
+    public String explainPlan()
+    {
+        return root.explainPlan();
+    }
+    
+}

Added: incubator/cassandra/src/org/apache/cassandra/cql/common/RowSourceDef.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/common/RowSourceDef.java?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/common/RowSourceDef.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/common/RowSourceDef.java Mon Mar  2 06:12:46 2009
@@ -0,0 +1,48 @@
+/**
+ * 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.common;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * The abstract notion of a row source definition. A row source
+ * is literally just anything that returns rows back.
+ * 
+ * The concrete implementations of row source might be things like a 
+ * column family row source, a "super column family" row source, 
+ * a table row source, etc.
+ *
+ * Note: Instances of sub-classes of this class are part of the "shared" 
+ * execution plan of CQL. And hence they should not contain any mutable
+ * (i.e. session specific) execution state. Mutable state, such a bind
+ * variable values (corresponding to say a rowKey or a column Key) are
+ * note part of the RowSourceDef tree.
+ * 
+ * [Eventually the notion of a "mutable" portion of the RowSource (RowSourceMut)
+ * will be introduced to hold session-specific execution state of the RowSource.
+ * For example, this would be needed when implementing iterator style rowsources
+ * that yields rows back one at a time as opposed to returning them in one
+ * shot.]
+ */
+public abstract class RowSourceDef
+{
+    public abstract List<Map<String,String>> getRows();
+    public abstract String explainPlan();  
+}
\ No newline at end of file

Added: incubator/cassandra/src/org/apache/cassandra/cql/common/SetColumnMap.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/common/SetColumnMap.java?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/common/SetColumnMap.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/common/SetColumnMap.java Mon Mar  2 06:12:46 2009
@@ -0,0 +1,133 @@
+/**
+ * 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.common;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.cql.execution.RuntimeErrorMsg;
+import org.apache.cassandra.db.RowMutation;
+import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.utils.LogUtil;
+import org.apache.log4j.Logger;
+import org.apache.cassandra.cql.execution.*;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.service.*;
+
+/**
+ * Execution plan for batch setting a set of columns in a Simple/Super column family.
+ *   SET table.standard_cf[<rowKey>] = <columnMapExpr>;                
+ *   SET table.super_cf[<rowKey>][<superColumn>] = <columnMapExpr>;
+ */
+public class SetColumnMap extends DMLPlan
+{
+    private final static Logger logger_ = Logger.getLogger(SetUniqueKey.class);    
+    private CFMetaData    cfMetaData_;
+    private OperandDef    rowKey_;
+    private OperandDef    superColumnKey_;
+    private ColumnMapExpr columnMapExpr_;
+
+    /**
+     *  construct an execution plan node to set the column map for a Standard Column Family.
+     *
+     *    SET table.standard_cf[<rowKey>] = <columnMapExpr>;                
+     */
+    public SetColumnMap(CFMetaData cfMetaData, OperandDef rowKey, ColumnMapExpr columnMapExpr)
+    {
+        cfMetaData_     = cfMetaData;
+        rowKey_         = rowKey;
+        superColumnKey_ = null;        
+        columnMapExpr_  = columnMapExpr;
+    }
+
+    /**
+     * Construct an execution plan node to set the column map for a Super Column Family
+     * 
+     *   SET table.super_cf[<rowKey>][<superColumn>] = <columnMapExpr>;
+     */
+    public SetColumnMap(CFMetaData cfMetaData, OperandDef rowKey, OperandDef superColumnKey, ColumnMapExpr columnMapExpr)
+    {
+        cfMetaData_     = cfMetaData;
+        rowKey_         = rowKey;
+        superColumnKey_ = superColumnKey;
+        columnMapExpr_  = columnMapExpr;
+    }
+
+    public CqlResult execute()
+    {
+        String columnFamily_column;
+        
+        if (superColumnKey_ != null)
+        {
+            String superColumnKey = (String)(superColumnKey_.get());
+            columnFamily_column = cfMetaData_.cfName + ":" + superColumnKey + ":";
+        }
+        else
+        {
+            columnFamily_column = cfMetaData_.cfName + ":";
+        }
+
+        try
+        {
+            RowMutation rm = new RowMutation(cfMetaData_.tableName, (String)(rowKey_.get()));
+            long time = System.currentTimeMillis();
+
+            for (Pair<OperandDef, OperandDef> entry : columnMapExpr_)
+            {
+                OperandDef columnKey = entry.getFirst();
+                OperandDef value     = entry.getSecond();
+
+                rm.add(columnFamily_column + (String)(columnKey.get()), ((String)value.get()).getBytes(), time);
+            }
+            StorageProxy.insert(rm);
+        }
+        catch (Exception e)
+        {
+            logger_.error(LogUtil.throwableToString(e));
+            throw new RuntimeException(RuntimeErrorMsg.GENERIC_ERROR.getMsg());            
+        }
+        return null;
+    }
+
+    public String explainPlan()
+    {
+        StringBuffer sb = new StringBuffer();
+        
+        String prefix =
+            String.format("%s Column Family: Batch SET a set of columns: \n" +
+            "   Table Name:     %s\n" +
+            "   Column Famly:   %s\n" +
+            "   RowKey:         %s\n" +
+            "%s",
+            cfMetaData_.columnType,
+            cfMetaData_.tableName,
+            cfMetaData_.cfName,
+            rowKey_.explain(),
+            (superColumnKey_ == null) ? "" : "   SuperColumnKey: " + superColumnKey_.explain() + "\n");                
+
+        for (Pair<OperandDef, OperandDef> entry : columnMapExpr_)
+        {
+            OperandDef columnKey = entry.getFirst();
+            OperandDef value     = entry.getSecond();
+            sb.append(String.format("   ColumnKey:        %s\n" +
+                                    "   Value:            %s\n",
+                                    columnKey.explain(), value.explain()));
+        }
+        
+        return prefix + sb.toString();
+    }
+}
\ No newline at end of file

Added: incubator/cassandra/src/org/apache/cassandra/cql/common/SetSuperColumnMap.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/common/SetSuperColumnMap.java?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/common/SetSuperColumnMap.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/common/SetSuperColumnMap.java Mon Mar  2 06:12:46 2009
@@ -0,0 +1,119 @@
+/**
+ * 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.common;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.cql.execution.RuntimeErrorMsg;
+import org.apache.cassandra.db.RowMutation;
+import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.utils.LogUtil;
+import org.apache.log4j.Logger;
+import org.apache.cassandra.cql.execution.*;
+
+/**
+ * Execution plan for batch setting a set of super columns in a Super column family.
+  *   SET table.super_cf[<rowKey>] = <superColumnMapExpr>;
+ */
+public class SetSuperColumnMap extends DMLPlan
+{
+    private final static Logger logger_ = Logger.getLogger(SetUniqueKey.class);    
+    private CFMetaData         cfMetaData_;
+    private OperandDef         rowKey_;
+    private SuperColumnMapExpr superColumnMapExpr_;
+
+    /**
+     *  construct an execution plan node to batch set a bunch of super columns in a 
+     *  super column family.
+     *
+     *    SET table.super_cf[<rowKey>] = <superColumnMapExpr>;
+     */
+    public SetSuperColumnMap(CFMetaData cfMetaData, OperandDef rowKey, SuperColumnMapExpr superColumnMapExpr)
+    {
+        cfMetaData_         = cfMetaData;
+        rowKey_             = rowKey;
+        superColumnMapExpr_ = superColumnMapExpr;
+    }
+    
+    public CqlResult execute()
+    {
+        try
+        {
+            RowMutation rm = new RowMutation(cfMetaData_.tableName, (String)(rowKey_.get()));
+            long time = System.currentTimeMillis();
+
+            for (Pair<OperandDef, ColumnMapExpr> superColumn : superColumnMapExpr_)
+            {
+                OperandDef    superColumnKey = superColumn.getFirst();
+                ColumnMapExpr columnMapExpr = superColumn.getSecond();
+                
+                String columnFamily_column = cfMetaData_.cfName + ":" + (String)(superColumnKey.get()) + ":";
+                
+                for (Pair<OperandDef, OperandDef> entry : columnMapExpr)
+                {
+                    OperandDef columnKey = entry.getFirst();
+                    OperandDef value     = entry.getSecond();
+                    rm.add(columnFamily_column + (String)(columnKey.get()), ((String)value.get()).getBytes(), time);
+                }
+            }
+            StorageProxy.insert(rm);
+        }
+        catch (Exception e)
+        {
+            logger_.error(LogUtil.throwableToString(e));
+            throw new RuntimeException(RuntimeErrorMsg.GENERIC_ERROR.getMsg());            
+        }
+        return null;
+
+    }
+
+    public String explainPlan()
+    {
+        StringBuffer sb = new StringBuffer();
+        
+        String prefix =
+            String.format("%s Column Family: Batch SET a set of Super Columns: \n" +
+            "   Table Name:     %s\n" +
+            "   Column Famly:   %s\n" +
+            "   RowKey:         %s\n",
+            cfMetaData_.columnType,
+            cfMetaData_.tableName,
+            cfMetaData_.cfName,
+            rowKey_.explain());
+
+        for (Pair<OperandDef, ColumnMapExpr> superColumn : superColumnMapExpr_)
+        {
+            OperandDef    superColumnKey = superColumn.getFirst();
+            ColumnMapExpr columnMapExpr = superColumn.getSecond();
+
+            for (Pair<OperandDef, OperandDef> entry : columnMapExpr)
+            {
+                OperandDef columnKey = entry.getFirst();
+                OperandDef value     = entry.getSecond();
+                sb.append(String.format("     SuperColumnKey: %s\n" + 
+                                        "     ColumnKey:      %s\n" +
+                                        "     Value:          %s\n",
+                                        superColumnKey.explain(),
+                                        columnKey.explain(),
+                                        value.explain()));
+            }
+        }
+        
+        return prefix + sb.toString();
+    }
+}
\ No newline at end of file

Added: incubator/cassandra/src/org/apache/cassandra/cql/common/SetUniqueKey.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/common/SetUniqueKey.java?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/common/SetUniqueKey.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/common/SetUniqueKey.java Mon Mar  2 06:12:46 2009
@@ -0,0 +1,118 @@
+/**
+ * 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.common;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.cql.execution.RuntimeErrorMsg;
+import org.apache.cassandra.db.RowMutation;
+import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.utils.LogUtil;
+import org.apache.log4j.Logger;
+
+
+/**
+ * Execution plan for setting a specific column in a Simple/Super column family.
+ *   SET table.standard_cf[<rowKey>][<columnKey>] = <value>;
+ *   SET table.super_cf[<rowKey>][<superColumnKey>][<columnKey>] = <value>; 
+ */
+public class SetUniqueKey extends DMLPlan
+{
+    private final static Logger logger_ = Logger.getLogger(SetUniqueKey.class);    
+    private CFMetaData cfMetaData_;
+    private OperandDef rowKey_;
+    private OperandDef superColumnKey_;
+    private OperandDef columnKey_;
+    private OperandDef value_;
+
+    /**
+     *  Construct an execution plan for setting a column in a simple column family
+     * 
+     *   SET table.standard_cf[<rowKey>][<columnKey>] = <value>;
+     */
+    public SetUniqueKey(CFMetaData cfMetaData, OperandDef rowKey, OperandDef columnKey, OperandDef value)
+    {
+        cfMetaData_     = cfMetaData;
+        rowKey_         = rowKey;
+        columnKey_      = columnKey;
+        superColumnKey_ = null;
+        value_          = value;
+    }
+    
+    /**
+     * Construct execution plan for setting a column in a super column family.
+     * 
+     *  SET table.super_cf[<rowKey>][<superColumnKey>][<columnKey>] = <value>;
+     */
+    public SetUniqueKey(CFMetaData cfMetaData, OperandDef rowKey, OperandDef superColumnKey, OperandDef columnKey, OperandDef value)
+    {
+        cfMetaData_     = cfMetaData;
+        rowKey_         = rowKey;
+        superColumnKey_ = superColumnKey;
+        columnKey_      = columnKey;
+        value_          = value;
+    }
+
+    public CqlResult execute()
+    {
+        String columnKey = (String)(columnKey_.get());
+        String columnFamily_column;
+
+        if (superColumnKey_ != null)
+        {
+            String superColumnKey = (String)(superColumnKey_.get());
+            columnFamily_column = cfMetaData_.cfName + ":" + superColumnKey + ":" + columnKey;
+        }
+        else
+        {
+            columnFamily_column = cfMetaData_.cfName + ":" + columnKey;
+        }
+
+        try
+        {
+            RowMutation rm = new RowMutation(cfMetaData_.tableName, (String)(rowKey_.get()));
+            rm.add(columnFamily_column, ((String)value_.get()).getBytes(), System.currentTimeMillis());
+            StorageProxy.insert(rm);
+        }
+        catch (Exception e)
+        {
+            logger_.error(LogUtil.throwableToString(e));
+            throw new RuntimeException(RuntimeErrorMsg.GENERIC_ERROR.getMsg());            
+        }
+        return null;
+    }
+
+    public String explainPlan()
+    {
+        return
+            String.format("%s Column Family: Unique Key SET: \n" +
+                "   Table Name:     %s\n" +
+                "   Column Famly:   %s\n" +
+                "   RowKey:         %s\n" +
+                "%s" +
+                "   ColumnKey:      %s\n" +
+                "   Value:          %s\n",
+                cfMetaData_.columnType,
+                cfMetaData_.tableName,
+                cfMetaData_.cfName,
+                rowKey_.explain(),
+                (superColumnKey_ == null) ? "" : "   SuperColumnKey: " + superColumnKey_.explain() + "\n",                
+                columnKey_.explain(),
+                value_.explain());
+    }
+}
\ No newline at end of file

Added: incubator/cassandra/src/org/apache/cassandra/cql/common/SuperColumnMapExpr.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/common/SuperColumnMapExpr.java?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/common/SuperColumnMapExpr.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/common/SuperColumnMapExpr.java Mon Mar  2 06:12:46 2009
@@ -0,0 +1,25 @@
+/* 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.common;
+
+import java.util.ArrayList;
+
+public class SuperColumnMapExpr extends ArrayList<Pair<OperandDef, ColumnMapExpr>>
+{
+    private static final long serialVersionUID = 1L;
+};
\ No newline at end of file

Added: incubator/cassandra/src/org/apache/cassandra/cql/common/SuperColumnRangeQueryRSD.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/common/SuperColumnRangeQueryRSD.java?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/common/SuperColumnRangeQueryRSD.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/common/SuperColumnRangeQueryRSD.java Mon Mar  2 06:12:46 2009
@@ -0,0 +1,132 @@
+/**
+ * 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.common;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.List;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.cql.execution.RuntimeErrorMsg;
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.IColumn;
+import org.apache.cassandra.db.Row;
+import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.service.column_t;
+import org.apache.cassandra.service.superColumn_t;
+import org.apache.cassandra.utils.LogUtil;
+import org.apache.log4j.Logger;
+import org.apache.cassandra.db.*;
+
+/**
+ * A Row Source Defintion (RSD) for doing a super column range query on a Super Column Family.
+ */
+public class SuperColumnRangeQueryRSD extends RowSourceDef
+{
+    private final static Logger logger_ = Logger.getLogger(SuperColumnRangeQueryRSD.class);
+    private CFMetaData cfMetaData_;
+    private OperandDef rowKey_;
+    private OperandDef superColumnKey_;
+    private int        offset_;
+    private int        limit_;
+
+    /**
+     * Set up a range query on super column map in a super column family.
+     * The super column map is identified by the rowKey.
+     * 
+     * Note: "limit" of -1 is the equivalent of no limit.
+     *       "offset" specifies the number of rows to skip.
+     *        An offset of 0 implies from the first row.
+     */
+    public SuperColumnRangeQueryRSD(CFMetaData cfMetaData, OperandDef rowKey, int offset, int limit)
+    {
+        cfMetaData_     = cfMetaData;
+        rowKey_         = rowKey;
+        offset_         = offset;
+        limit_          = limit;
+    }
+
+    public List<Map<String,String>> getRows()
+    {
+        Row row = null;
+        try
+        {
+            String key = (String)(rowKey_.get());
+            row = StorageProxy.readProtocol(cfMetaData_.tableName, key, cfMetaData_.cfName,
+                                            offset_, limit_, StorageService.ConsistencyLevel.WEAK);
+        }
+        catch (Exception e)
+        {
+            logger_.error(LogUtil.throwableToString(e));
+            throw new RuntimeException(RuntimeErrorMsg.GENERIC_ERROR.getMsg());
+        }
+
+        List<Map<String, String>> rows = new LinkedList<Map<String, String>>();
+        if (row != null)
+        {
+            Map<String, ColumnFamily> cfMap = row.getColumnFamilies();
+            if (cfMap != null && cfMap.size() > 0)
+            {
+                ColumnFamily cfamily = cfMap.get(cfMetaData_.cfName);
+                if (cfamily != null)
+                {
+                    Collection<IColumn> columns = cfamily.getAllColumns();
+                    if (columns != null && columns.size() > 0)
+                    {
+                        for (IColumn column : columns)
+                        {
+                            Collection<IColumn> subColumns = column.getSubColumns();
+                            for( IColumn subColumn : subColumns )
+                            {
+                               Map<String, String> result = new HashMap<String, String>();
+                               result.put(cfMetaData_.n_superColumnKey, column.name());                               
+                               result.put(cfMetaData_.n_columnKey, subColumn.name());
+                               result.put(cfMetaData_.n_columnValue, new String(subColumn.value()));
+                               result.put(cfMetaData_.n_columnTimestamp, Long.toString(subColumn.timestamp()));
+                               rows.add(result);
+                            }
+                        }
+                    }
+                }
+            }
+        }
+        return rows;
+    }
+
+    public String explainPlan()
+    {
+        return String.format("%s Column Family: Super Column Range Query: \n" +
+                "  Table Name:       %s\n" +
+                "  Column Family:    %s\n" +
+                "  RowKey:           %s\n" +
+                "  Offset:           %d\n" +
+                "  Limit:            %d\n" +
+                "  Order By:         %s",
+                cfMetaData_.columnType,
+                cfMetaData_.tableName,
+                cfMetaData_.cfName,
+                rowKey_.explain(),
+                offset_, limit_,
+                cfMetaData_.indexProperty_);
+    }
+}
\ No newline at end of file

Added: incubator/cassandra/src/org/apache/cassandra/cql/common/UniqueKeyQueryRSD.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/common/UniqueKeyQueryRSD.java?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/common/UniqueKeyQueryRSD.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/common/UniqueKeyQueryRSD.java Mon Mar  2 06:12:46 2009
@@ -0,0 +1,165 @@
+/**
+ * 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.common;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.List;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.cql.execution.RuntimeErrorMsg;
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.IColumn;
+import org.apache.cassandra.db.Row;
+import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.LogUtil;
+import org.apache.log4j.Logger;
+import org.apache.cassandra.db.*;
+
+/**
+ * A Row Source Defintion (RSD) for looking up a unique column within a column family.
+ */
+public class UniqueKeyQueryRSD extends RowSourceDef
+{
+    private final static Logger logger_ = Logger.getLogger(UniqueKeyQueryRSD.class);    
+    private CFMetaData cfMetaData_;
+    private OperandDef rowKey_;
+    private OperandDef superColumnKey_;
+    private OperandDef columnKey_;
+
+    // super column family
+    public UniqueKeyQueryRSD(CFMetaData cfMetaData, OperandDef rowKey, OperandDef superColumnKey, OperandDef columnKey)
+    {
+        cfMetaData_     = cfMetaData;
+        rowKey_         = rowKey;
+        superColumnKey_ = superColumnKey;
+        columnKey_      = columnKey;
+    }
+
+    // simple column family
+    public UniqueKeyQueryRSD(CFMetaData cfMetaData, OperandDef rowKey, OperandDef columnKey)
+    {
+        cfMetaData_ = cfMetaData;
+        rowKey_     = rowKey;
+        columnKey_  = columnKey;
+        superColumnKey_ = null;
+    }
+
+    // specific column lookup
+    public List<Map<String,String>> getRows() throws RuntimeException
+    {
+        String columnKey = (String)(columnKey_.get());
+        String columnFamily_column;
+        String superColumnKey = null;
+
+        if (superColumnKey_ != null)
+        {
+            superColumnKey = (String)(superColumnKey_.get());
+            columnFamily_column = cfMetaData_.cfName + ":" + superColumnKey + ":" + columnKey;
+        }
+        else
+        {
+            columnFamily_column = cfMetaData_.cfName + ":" + columnKey;
+        }
+
+        Row row = null;
+        try
+        {
+            String key = (String)(rowKey_.get());
+            row = StorageProxy.readProtocol(cfMetaData_.tableName, key, columnFamily_column, -1,
+                                            Integer.MAX_VALUE, StorageService.ConsistencyLevel.WEAK);
+        }
+        catch (Exception e)
+        {
+            logger_.error(LogUtil.throwableToString(e));
+            throw new RuntimeException(RuntimeErrorMsg.GENERIC_ERROR.getMsg());
+        }
+
+        if (row != null)
+        {
+            Map<String, ColumnFamily> cfMap = row.getColumnFamilies();
+            if (cfMap != null && cfMap.size() > 0)
+            {
+                ColumnFamily cfamily = cfMap.get(cfMetaData_.cfName);
+                if (cfamily != null)
+                {
+                    Collection<IColumn> columns = null;
+                    if (superColumnKey_ != null)
+                    {
+                        // this is the super column case 
+                        IColumn column = cfamily.getColumn(superColumnKey);
+                        if (column != null)
+                            columns = column.getSubColumns();
+                    }
+                    else
+                    {
+                        columns = cfamily.getAllColumns();
+                    }
+                    
+                    if (columns != null && columns.size() > 0)
+                    {
+                        if (columns.size() > 1)
+                        {
+                            // We are looking up by a rowKey & columnKey. There should
+                            // be at most one column that matches. If we find more than
+                            // one, then it is an internal error.
+                            throw new RuntimeException(RuntimeErrorMsg.INTERNAL_ERROR.getMsg("Too many columns found for: " + columnKey));
+                        }
+                        for (IColumn column : columns)
+                        {
+                            List<Map<String, String>> rows = new LinkedList<Map<String, String>>();
+
+                            Map<String, String> result = new HashMap<String, String>();
+                            result.put(cfMetaData_.n_columnKey, column.name());
+                            result.put(cfMetaData_.n_columnValue, new String(column.value()));
+                            result.put(cfMetaData_.n_columnTimestamp, Long.toString(column.timestamp()));
+                            
+                            rows.add(result);
+                                
+                            // at this point, due to the prior checks, we are guaranteed that
+                            // there is only one item in "columns".
+                            return rows;
+                        }
+                        return null;
+                    }
+                }
+            }
+        }
+        throw new RuntimeException(RuntimeErrorMsg.NO_DATA_FOUND.getMsg());
+    }
+
+    public String explainPlan()
+    {
+        return String.format("%s Column Family: Unique Key Query: \n" +
+                "   Table Name:     %s\n" +
+                "   Column Famly:   %s\n" +
+                "   RowKey:         %s\n" +
+                "%s" +
+                "   ColumnKey:      %s",
+                cfMetaData_.columnType,
+                cfMetaData_.tableName,
+                cfMetaData_.cfName,
+                rowKey_.explain(),
+                (superColumnKey_ == null) ? "" : "   SuperColumnKey: " + superColumnKey_.explain() + "\n",                
+                columnKey_.explain());
+    }
+}
\ No newline at end of file

Added: incubator/cassandra/src/org/apache/cassandra/cql/common/Utils.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/common/Utils.java?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/common/Utils.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/common/Utils.java Mon Mar  2 06:12:46 2009
@@ -0,0 +1,62 @@
+/**
+ * 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.common;
+
+public class Utils
+{
+    /*
+     * Strips leading and trailing "'" characters, and handles
+     * and escaped characters such as \n, \r, etc.
+     * [Shameless clone from hive.]
+     */
+    public static String unescapeSQLString(String b) 
+    {
+        assert(b.charAt(0) == '\'');
+        assert(b.charAt(b.length()-1) == '\'');
+        StringBuilder sb = new StringBuilder(b.length());
+        
+        for (int i=1; i+1<b.length(); i++)
+        {
+            if (b.charAt(i) == '\\' && i+2<b.length())
+            {
+                char n=b.charAt(i+1);
+                switch(n)
+                {
+                case '0': sb.append("\0"); break;
+                case '\'': sb.append("'"); break;
+                case '"': sb.append("\""); break;
+                case 'b': sb.append("\b"); break;
+                case 'n': sb.append("\n"); break;
+                case 'r': sb.append("\r"); break;
+                case 't': sb.append("\t"); break;
+                case 'Z': sb.append("\u001A"); break;
+                case '\\': sb.append("\\"); break;
+                case '%': sb.append("%"); break;
+                case '_': sb.append("_"); break;
+                default: sb.append(n);
+                }
+            } 
+            else
+            {
+                sb.append(b.charAt(i));
+            }
+        }
+        return sb.toString();
+    } 
+}

Added: incubator/cassandra/src/org/apache/cassandra/cql/compiler/common/CompilerErrorMsg.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/compiler/common/CompilerErrorMsg.java?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/compiler/common/CompilerErrorMsg.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/compiler/common/CompilerErrorMsg.java Mon Mar  2 06:12:46 2009
@@ -0,0 +1,70 @@
+/**
+ * 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.compiler.common;
+
+import org.antlr.runtime.tree.CommonTree;
+
+/**
+ * List of error messages thrown by the CQL Compiler
+ **/
+
+public enum CompilerErrorMsg
+{
+    // Error messages with String.format() style format specifiers
+    GENERIC_ERROR("CQL Compilation Error"),
+    INTERNAL_ERROR("CQL Compilation Internal Error"),
+    INVALID_TABLE("Table '%s' does not exist"),
+    INVALID_COLUMN_FAMILY("Column Family '%s' not found in table '%s'"),
+    TOO_MANY_DIMENSIONS("Too many dimensions specified for %s Column Family"),
+    INVALID_TYPE("Expression is of invalid type")
+    ;
+
+    private String mesg;
+    CompilerErrorMsg(String mesg)
+    {
+        this.mesg = mesg;
+    }
+    
+    private static String getLineAndPosition(CommonTree tree) 
+    {
+        if (tree.getChildCount() == 0)
+        {
+            return tree.getToken().getLine() + ":" + tree.getToken().getCharPositionInLine();
+        }
+        return getLineAndPosition((CommonTree)tree.getChild(0));
+    }
+
+    // Returns the formatted error message. Derives line/position information
+    // from the "tree" node passed in.
+    public String getMsg(CommonTree tree, Object... args)
+    {
+        // We allocate another array since we want to add line and position as an 
+        // implicit additional first argument to pass on to String.format.
+        Object[] newArgs = new Object[args.length + 1];
+        newArgs[0] = getLineAndPosition(tree);
+        System.arraycopy(args, 0, newArgs, 1, args.length);
+
+        // note: mesg itself might contain other format specifiers...
+        return String.format("line %s " + mesg, newArgs);
+    } 
+
+    String getMsg()
+    {
+        return mesg;
+    } 
+}

Added: incubator/cassandra/src/org/apache/cassandra/cql/compiler/common/CqlCompiler.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/compiler/common/CqlCompiler.java?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/compiler/common/CqlCompiler.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/compiler/common/CqlCompiler.java Mon Mar  2 06:12:46 2009
@@ -0,0 +1,149 @@
+/**
+ * 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.compiler.common;
+
+
+import org.apache.cassandra.cql.common.*;
+import org.apache.cassandra.cql.compiler.parse.*;
+import org.apache.cassandra.cql.compiler.sem.*;
+
+import java.util.ArrayList;
+
+import org.antlr.runtime.*;
+import org.antlr.runtime.tree.*;
+import org.apache.cassandra.cql.common.Plan;
+import org.apache.cassandra.cql.compiler.parse.CqlLexer;
+import org.apache.cassandra.cql.compiler.parse.CqlParser;
+import org.apache.cassandra.cql.compiler.parse.ParseError;
+import org.apache.cassandra.cql.compiler.parse.ParseException;
+import org.apache.cassandra.cql.compiler.sem.SemanticException;
+import org.apache.cassandra.cql.compiler.sem.SemanticPhase;
+
+public class CqlCompiler
+{
+    // ANTLR does not provide case-insensitive tokenization support
+    // out of the box. So we override the LA (lookahead) function
+    // of the ANTLRStringStream class. Note: This doesn't change the
+    // token text-- but just relaxes the matching rules to match
+    // in upper case. [Logic borrowed from Hive code.]
+    // 
+    // Also see discussion on this topic in:
+    // http://www.antlr.org/wiki/pages/viewpage.action?pageId=1782.
+    public class ANTLRNoCaseStringStream  extends ANTLRStringStream
+    {
+        public ANTLRNoCaseStringStream(String input)
+        {
+            super(input);
+        }
+    
+        public int LA(int i)
+        {
+            int returnChar = super.LA(i);
+            if (returnChar == CharStream.EOF)
+            {
+                return returnChar; 
+            }
+            else if (returnChar == 0) 
+            {
+                return returnChar;
+            }
+
+            return Character.toUpperCase((char)returnChar);
+        }
+    }
+
+    // Override CQLParser. This gives flexibility in altering default error
+    // messages as well as accumulating multiple errors.
+    public class CqlParserX extends CqlParser
+    {
+        private ArrayList<ParseError> errors;
+
+        public CqlParserX(TokenStream input)
+        {
+            super(input);
+            errors = new ArrayList<ParseError>();
+        }
+
+        protected void mismatch(IntStream input, int ttype, BitSet follow) throws RecognitionException
+        {
+            throw new MismatchedTokenException(ttype, input);
+        }
+
+        public void recoverFromMismatchedSet(IntStream input,
+                                             RecognitionException re,
+                                             BitSet follow) throws RecognitionException
+        {
+            throw re;
+        }
+
+        public void displayRecognitionError(String[] tokenNames,
+                                            RecognitionException e)
+        {
+            errors.add(new ParseError(this, e, tokenNames));
+        }
+
+        public ArrayList<ParseError> getErrors()
+        {
+            return errors;
+        }
+    }
+
+    // Compile a CQL query
+    public Plan compileQuery(String query) throws ParseException, SemanticException
+    {
+        CommonTree queryTree = null;
+        CqlLexer lexer = null;
+        CqlParserX parser = null;
+        CommonTokenStream tokens = null;
+
+        ANTLRStringStream input = new ANTLRNoCaseStringStream(query);
+
+        lexer = new CqlLexer(input);
+        tokens = new CommonTokenStream(lexer);
+        parser = new CqlParserX(tokens);
+
+        // built AST
+        try
+        {
+            queryTree = (CommonTree)(parser.root().getTree());
+        }
+        catch (RecognitionException e)
+        {
+            throw new ParseException(parser.getErrors());            
+        }
+        catch (RewriteEmptyStreamException e)
+        {
+            throw new ParseException(parser.getErrors());            
+        }
+
+        if (!parser.getErrors().isEmpty())
+        {
+            throw new ParseException(parser.getErrors());
+        }
+
+        if (!parser.errors.isEmpty())
+        {
+            throw new ParseException("parser error");
+        }
+
+        // Semantic analysis and code-gen.
+        // Eventually, I anticipate, I'll be forking these off into two separate phases.
+        return SemanticPhase.doSemanticAnalysis(queryTree);
+    }
+}

Added: incubator/cassandra/src/org/apache/cassandra/cql/compiler/parse/Cql.g
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/compiler/parse/Cql.g?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/compiler/parse/Cql.g (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/compiler/parse/Cql.g Mon Mar  2 06:12:46 2009
@@ -0,0 +1,308 @@
+/**
+ * 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.
+ */
+
+// ANTLR Grammar Definition for Cassandra Query Language (CQL)
+//
+// CQL is a query language tailored for Cassandra's multi-level (or 
+// nested-table like) data model where values stored for each key 
+// can be:
+//
+//  * a simple column map (a 1-level nested table). This is the case
+//    for a simple column family.
+//
+//  or,
+// 
+//  * a supercolumn column map, which in turn contains a column map
+//    per super column (i.e. a 2-level nested table). This is the case
+//    for a super column family.
+//
+// For the common case of key-based data retrieval or storage, CQL
+// provides array like get/set syntax, such as:
+//
+//        SET user.profile['99']['name'] = 'joe';
+//        SET user.profile['99']['age'] = '27';
+//        GET user.profile['99']['name'];
+//        GET user.profile['99'];
+//
+// When additional constraints need to be applied to data being retrieved
+// (such as imposing row limits, retrieving counts, retrieving data for
+// a subset of super columns or columns and son on) CQL falls back to more
+// traditional SQL-like syntax. 
+//
+// *Note*: The SQL syntax supported by CQL doesn't support the full
+// relational algebra. For example, it doesn't have any support for
+// joins. It also imposes restrictions on the types of filters and ORDER
+// BY clauses it supports-- generally only those queries that can be
+// efficiently answered based on data layout are supported. Suppose a column
+// family has been configured to store columns in time sorted fashion,
+// CQL will not support 'ORDER BY column_name' for such a column family. 
+//
+
+
+//
+// NOTE: The grammar is in a very rudimentary/prototypish shape right now.
+//       Will undergo fairly big restructuring in the next checkin.
+//
+
+grammar Cql;
+
+options {
+    output=AST;
+    ASTLabelType=CommonTree;
+    backtrack=true;
+}
+
+//
+// AST Nodes. We use a A_ prefix convention for these AST node names.
+//
+tokens {
+
+    // Top-level AST nodes
+    // These typically correspond to various top-level CQL statements.
+    A_DELETE;
+    A_GET;
+    A_SELECT;
+    A_SET;
+    A_EXPLAIN_PLAN;
+
+    // Internal AST nodes
+    A_COLUMN_ACCESS;
+    A_COLUMN_MAP_ENTRY;
+    A_COLUMN_MAP_VALUE;
+    A_FROM;
+    A_KEY_IN_LIST;
+    A_KEY_EXACT_MATCH;
+    A_LIMIT;
+    A_OFFSET;
+    A_ORDER_BY;
+    A_SUPERCOLUMN_MAP_ENTRY;
+    A_SUPERCOLUMN_MAP_VALUE;
+    A_SELECT_CLAUSE;
+    A_WHERE;
+}
+
+@parser::header {
+            package com.facebook.infrastructure.cql.compiler.parse;
+        }
+
+@lexer::header {
+            package com.facebook.infrastructure.cql.compiler.parse;
+        }
+
+//
+// Parser Section
+//
+
+// the root node
+root
+    : stmt SEMICOLON? EOF -> stmt
+    | K_EXPLAIN K_PLAN stmt SEMICOLON? EOF -> ^(A_EXPLAIN_PLAN stmt)
+    ;
+
+stmt
+    : deleteStmt
+    | getStmt
+    | selectStmt
+    | setStmt
+    ;
+
+getStmt
+    : K_GET columnSpec  -> ^(A_GET columnSpec)
+    ;
+
+setStmt
+    : K_SET columnSpec '=' valueExpr -> ^(A_SET columnSpec valueExpr)
+    ;
+
+selectStmt
+    : selectClause
+        fromClause?
+        whereClause?
+        limitClause? -> ^(A_SELECT selectClause fromClause? whereClause? limitClause?)
+    ;
+
+selectClause
+	: K_SELECT selectList -> ^(A_SELECT_CLAUSE selectList)
+	;
+
+selectList
+	: selectListItem (',' selectListItem)*
+	;
+
+selectListItem
+	: columnExpression
+	| '(' selectStmt ')' -> ^(A_SELECT selectStmt)
+	; 
+
+columnExpression
+	: columnOrSuperColumnName columnExpressionRest;
+	
+columnExpressionRest
+	:  /* empty */
+	|  '[' stringVal ']' columnExpressionRest
+	;
+
+tableExpression
+    : tableName '.' columnFamilyName '[' stringVal ']'; 
+
+fromClause
+    : K_FROM tableExpression -> ^(A_FROM tableExpression)
+    ;
+
+whereClause
+    : K_WHERE keyInClause   -> ^(A_WHERE keyInClause)
+    | K_WHERE keyExactMatch -> ^(A_WHERE keyExactMatch)
+    ;
+
+keyInClause
+    : columnOrSuperColumnName K_IN '(' a+=stringVal (',' a+=stringVal)* ')'
+    	-> ^(A_KEY_IN_LIST columnOrSuperColumnName $a+)
+    ;
+
+keyExactMatch
+    : columnOrSuperColumnName '=' stringVal
+    	-> ^(A_KEY_EXACT_MATCH columnOrSuperColumnName stringVal)
+    ;
+
+limitClause
+    : K_LIMIT IntegerLiteral -> ^(A_LIMIT IntegerLiteral);  
+
+deleteStmt
+    : K_DELETE columnSpec -> ^(A_DELETE columnSpec)
+    ;
+
+columnSpec
+    : tableName '.' columnFamilyName '[' rowKey ']' 
+        ( '[' a+=columnOrSuperColumnKey ']' 
+            ('[' a+=columnOrSuperColumnKey ']')? 
+        )?
+        -> ^(A_COLUMN_ACCESS tableName columnFamilyName rowKey ($a+)?)
+    ;
+
+tableName: Identifier;
+
+columnFamilyName: Identifier;
+
+valueExpr
+   :  cellValue
+   |  columnMapValue
+   |  superColumnMapValue
+   ;
+
+cellValue
+   : stringVal;
+
+columnMapValue
+   : LEFT_BRACE columnMapEntry (COMMA columnMapEntry)* RIGHT_BRACE
+     -> ^(A_COLUMN_MAP_VALUE columnMapEntry+)
+   ;
+
+superColumnMapValue
+   : LEFT_BRACE superColumnMapEntry (COMMA superColumnMapEntry)* RIGHT_BRACE
+     -> ^(A_SUPERCOLUMN_MAP_VALUE superColumnMapEntry+)
+   ;
+
+columnMapEntry
+   : columnKey ASSOC cellValue -> ^(A_COLUMN_MAP_ENTRY columnKey cellValue)
+   ;
+
+superColumnMapEntry
+   : superColumnKey ASSOC columnMapValue -> ^(A_SUPERCOLUMN_MAP_ENTRY superColumnKey columnMapValue)
+   ;
+
+columnOrSuperColumnName: Identifier; 
+
+rowKey:                  stringVal;
+columnOrSuperColumnKey:  stringVal;
+columnKey:               stringVal;
+superColumnKey:          stringVal;
+
+// String Values can either be query params (aka bind variables)
+// or string literals.
+stringVal
+    : '?'                   // bind variable
+    | StringLiteral         // 
+    ;
+
+//
+// Lexer Section
+//
+
+// Keywords (in alphabetical order for convenience)
+K_BY:        'BY';
+K_DELETE:    'DELETE';
+K_EXPLAIN:   'EXPLAIN';
+K_FROM:      'FROM';
+K_GET:       'GET';
+K_IN:        'IN';
+K_LIMIT:     'LIMIT';
+K_OFFSET:    'OFFSET';
+K_ORDER:     'ORDER';
+K_PLAN:      'PLAN';
+K_SELECT:    'SELECT';
+K_SET:       'SET';
+K_WHERE:     'WHERE';
+
+// private syntactic rules
+fragment
+Letter
+    : 'a'..'z' 
+    | 'A'..'Z'
+    ;
+
+fragment
+Digit
+    : '0'..'9'
+    ;
+
+// syntactic Elements
+Identifier
+    : Letter ( Letter | Digit | '_')*
+    ;
+
+//
+// Literals 
+//
+
+// strings: escape single quote ' by repeating it '' (SQL style)
+StringLiteral
+    : '\'' (~'\'')* '\'' ( '\'' (~'\'')* '\'' )* 
+    ;
+
+// integer literals    
+IntegerLiteral
+    : Digit+
+    ;
+
+//
+// miscellaneous syntactic elements
+//
+WS
+    :  (' '|'\r'|'\t'|'\n') {skip();}  // whitepace
+    ;
+
+COMMENT 
+    : '--' (~('\n'|'\r'))*                     { $channel=HIDDEN; }
+    | '/*' (options {greedy=false;} : .)* '*/' { $channel=HIDDEN; }
+    ;
+
+ASSOC:        '=>';
+COMMA:        ',';
+LEFT_BRACE:   '{';
+RIGHT_BRACE:  '}';
+SEMICOLON:    ';';

Added: incubator/cassandra/src/org/apache/cassandra/cql/compiler/parse/Cql.tokens
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/cql/compiler/parse/Cql.tokens?rev=749205&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/cql/compiler/parse/Cql.tokens (added)
+++ incubator/cassandra/src/org/apache/cassandra/cql/compiler/parse/Cql.tokens Mon Mar  2 06:12:46 2009
@@ -0,0 +1,51 @@
+K_EXPLAIN=23
+K_OFFSET=41
+K_GET=25
+K_DELETE=33
+A_KEY_EXACT_MATCH=14
+K_BY=40
+A_SELECT=6
+A_SUPERCOLUMN_MAP_VALUE=19
+K_SELECT=27
+K_LIMIT=31
+Identifier=34
+K_SET=26
+K_WHERE=29
+COMMA=36
+A_EXPLAIN_PLAN=8
+A_LIMIT=15
+COMMENT=46
+K_ORDER=42
+RIGHT_BRACE=37
+A_COLUMN_MAP_VALUE=11
+SEMICOLON=22
+K_IN=30
+Digit=44
+A_OFFSET=16
+A_WHERE=21
+K_PLAN=24
+A_ORDER_BY=17
+K_FROM=28
+StringLiteral=39
+A_COLUMN_MAP_ENTRY=10
+WS=45
+A_FROM=12
+A_GET=5
+LEFT_BRACE=35
+A_KEY_IN_LIST=13
+A_COLUMN_ACCESS=9
+A_SUPERCOLUMN_MAP_ENTRY=18
+IntegerLiteral=32
+ASSOC=38
+A_SELECT_CLAUSE=20
+Letter=43
+A_DELETE=4
+A_SET=7
+'?'=53
+'='=47
+'('=48
+'['=50
+','=36
+'.'=52
+')'=49
+']'=51