You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ty...@apache.org on 2014/07/15 19:16:39 UTC

[1/2] git commit: Include snippet of query in SyntaxError messages

Repository: cassandra
Updated Branches:
  refs/heads/trunk 3966935b4 -> 8a5a82b8a


Include snippet of query in SyntaxError messages

Patch by Benjamin Lerer; reviewed by Tyler Hobbs for CASSANDRA-7111


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

Branch: refs/heads/trunk
Commit: 8a2d8a138d5ac1fc38b24bc7fd7513f6dbe86b24
Parents: 52c6dfb
Author: blerer <b_...@hotmail.com>
Authored: Tue Jul 15 12:05:47 2014 -0500
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Tue Jul 15 12:05:47 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 src/java/org/apache/cassandra/cql3/Cql.g        |  41 ++--
 .../apache/cassandra/cql3/ErrorCollector.java   | 242 +++++++++++++++++++
 .../apache/cassandra/cql3/ErrorListener.java    |  44 ++++
 .../apache/cassandra/cql3/QueryProcessor.java   |  11 +-
 .../apache/cassandra/cql3/CqlParserTest.java    |  90 +++++++
 .../cassandra/cql3/ErrorCollectorTest.java      | 218 +++++++++++++++++
 7 files changed, 621 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8a2d8a13/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 1781ef3..89f9b7c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -6,6 +6,7 @@
  * Adjust MT depth based on # of partition validating (CASSANDRA-5263)
  * Optimise NativeCell comparisons (CASSANDRA-6755)
  * Configurable client timeout for cqlsh (CASSANDRA-7516)
+ * Include snippet of CQL query near syntax error in messages (CASSANDRA-7111)
 
 
 2.1.0-final

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8a2d8a13/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index 948cd5e..9dcc268 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -51,7 +51,7 @@ options {
 }
 
 @members {
-    private final List<String> recognitionErrors = new ArrayList<String>();
+    private final List<ErrorListener> listeners = new ArrayList<ErrorListener>();
     private final List<ColumnIdentifier> bindVariables = new ArrayList<ColumnIdentifier>();
 
     public static final Set<String> reservedTypeNames = new HashSet<String>()
@@ -94,27 +94,26 @@ options {
         return marker;
     }
 
-    public void displayRecognitionError(String[] tokenNames, RecognitionException e)
+    public void addErrorListener(ErrorListener listener)
     {
-        String hdr = getErrorHeader(e);
-        String msg = getErrorMessage(e, tokenNames);
-        recognitionErrors.add(hdr + " " + msg);
+        this.listeners.add(listener);
     }
 
-    public void addRecognitionError(String msg)
+    public void removeErrorListener(ErrorListener listener)
     {
-        recognitionErrors.add(msg);
+        this.listeners.remove(listener);
     }
 
-    public List<String> getRecognitionErrors()
+    public void displayRecognitionError(String[] tokenNames, RecognitionException e)
     {
-        return recognitionErrors;
+        for (int i = 0, m = listeners.size(); i < m; i++)
+            listeners.get(i).syntaxError(this, tokenNames, e);
     }
 
-    public void throwLastRecognitionError() throws SyntaxException
+    private void addRecognitionError(String msg)
     {
-        if (recognitionErrors.size() > 0)
-            throw new SyntaxException(recognitionErrors.get((recognitionErrors.size()-1)));
+        for (int i = 0, m = listeners.size(); i < m; i++)
+            listeners.get(i).syntaxError(this, msg);
     }
 
     public Map<String, String> convertPropertyMap(Maps.Literal map)
@@ -189,24 +188,22 @@ options {
         return tokens.remove(0);
     }
 
-    private List<String> recognitionErrors = new ArrayList<String>();
+    private final List<ErrorListener> listeners = new ArrayList<ErrorListener>();
 
-    public void displayRecognitionError(String[] tokenNames, RecognitionException e)
+    public void addErrorListener(ErrorListener listener)
     {
-        String hdr = getErrorHeader(e);
-        String msg = getErrorMessage(e, tokenNames);
-        recognitionErrors.add(hdr + " " + msg);
+        this.listeners.add(listener);
     }
 
-    public List<String> getRecognitionErrors()
+    public void removeErrorListener(ErrorListener listener)
     {
-        return recognitionErrors;
+        this.listeners.remove(listener);
     }
 
-    public void throwLastRecognitionError() throws SyntaxException
+    public void displayRecognitionError(String[] tokenNames, RecognitionException e)
     {
-        if (recognitionErrors.size() > 0)
-            throw new SyntaxException(recognitionErrors.get((recognitionErrors.size()-1)));
+        for (int i = 0, m = listeners.size(); i < m; i++)
+            listeners.get(i).syntaxError(this, tokenNames, e);
     }
 }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8a2d8a13/src/java/org/apache/cassandra/cql3/ErrorCollector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ErrorCollector.java b/src/java/org/apache/cassandra/cql3/ErrorCollector.java
new file mode 100644
index 0000000..41536f5
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/ErrorCollector.java
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import java.util.LinkedList;
+
+import org.antlr.runtime.BaseRecognizer;
+import org.antlr.runtime.Parser;
+import org.antlr.runtime.RecognitionException;
+import org.antlr.runtime.Token;
+import org.antlr.runtime.TokenStream;
+import org.apache.cassandra.exceptions.SyntaxException;
+
+/**
+ * <code>ErrorListener</code> that collect and enhance the errors send by the CQL lexer and parser.
+ */
+public final class ErrorCollector implements ErrorListener
+{
+    /**
+     * The offset of the first token of the snippet.
+     */
+    private static final int FIRST_TOKEN_OFFSET = 10;
+
+    /**
+     * The offset of the last token of the snippet.
+     */
+    private static final int LAST_TOKEN_OFFSET = 2;
+
+    /**
+     * The CQL query.
+     */
+    private final String query;
+
+    /**
+     * The error messages.
+     */
+    private final LinkedList<String> errorMsgs = new LinkedList<>();
+
+    /**
+     * Creates a new <code>ErrorCollector</code> instance to collect the syntax errors associated to the specified CQL
+     * query.
+     *
+     * @param query the CQL query that will be parsed
+     */
+    public ErrorCollector(String query)
+    {
+        this.query = query;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public void syntaxError(BaseRecognizer recognizer, String[] tokenNames, RecognitionException e)
+    {
+        String hdr = recognizer.getErrorHeader(e);
+        String msg = recognizer.getErrorMessage(e, tokenNames);
+
+        StringBuilder builder = new StringBuilder().append(hdr)
+                .append(' ')
+                .append(msg);
+
+        if (recognizer instanceof Parser)
+            appendQuerySnippet((Parser) recognizer, builder);
+
+        errorMsgs.add(builder.toString());
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public void syntaxError(BaseRecognizer recognizer, String errorMsg)
+    {
+        errorMsgs.add(errorMsg);
+    }
+
+    /**
+     * Throws the last syntax error found by the lexer or the parser if it exists.
+     *
+     * @throws SyntaxException the syntax error.
+     */
+    public void throwLastSyntaxError() throws SyntaxException
+    {
+        if (!errorMsgs.isEmpty())
+            throw new SyntaxException(errorMsgs.getLast());
+    }
+
+    /**
+     * Appends a query snippet to the message to help the user to understand the problem.
+     *
+     * @param parser the parser used to parse the query
+     * @param builder the <code>StringBuilder</code> used to build the error message
+     */
+    private void appendQuerySnippet(Parser parser, StringBuilder builder)
+    {
+        TokenStream tokenStream = parser.getTokenStream();
+        int index = tokenStream.index();
+        int size = tokenStream.size();
+
+        Token from = tokenStream.get(getSnippetFirstTokenIndex(index));
+        Token to = tokenStream.get(getSnippetLastTokenIndex(index, size));
+        Token offending = tokenStream.get(index);
+
+        appendSnippet(builder, from, to, offending);
+    }
+
+    /**
+     * Appends a query snippet to the message to help the user to understand the problem.
+     *
+     * @param from the first token to include within the snippet
+     * @param to the last token to include within the snippet
+     * @param offending the token which is responsible for the error
+     */
+    final void appendSnippet(StringBuilder builder,
+                             Token from,
+                             Token to,
+                             Token offending)
+    {
+        String[] lines = query.split("\n");
+
+        boolean includeQueryStart = (from.getLine() == 1) && (from.getCharPositionInLine() == 0);
+        boolean includeQueryEnd = (to.getLine() == lines.length)
+                && (getLastCharPositionInLine(to) == lines[lines.length - 1].length());
+
+        builder.append(" (");
+
+        if (!includeQueryStart)
+            builder.append("...");
+
+        lines[lineIndex(to)] = lines[lineIndex(to)].substring(0, getLastCharPositionInLine(to));
+        lines[lineIndex(offending)] = highlightToken(lines[lineIndex(offending)], offending);
+        lines[lineIndex(from)] = lines[lineIndex(from)].substring(from.getCharPositionInLine());
+
+        for (int i = lineIndex(from), m = lineIndex(to); i <= m; i++)
+            builder.append(lines[i]);
+
+        if (!includeQueryEnd)
+            builder.append("...");
+
+        builder.append(")");
+    }
+
+    /**
+     * Puts the specified token within square brackets.
+     *
+     * @param line the line containing the token
+     * @param token the token to put within square brackets
+     */
+    private static String highlightToken(String line, Token token)
+    {
+        String newLine = insertChar(line, getLastCharPositionInLine(token), ']');
+        return insertChar(newLine, token.getCharPositionInLine(), '[');
+    }
+
+    /**
+     * Returns the index of the last character relative to the beginning of the line 0..n-1
+     *
+     * @param token the token
+     * @return the index of the last character relative to the beginning of the line 0..n-1
+     */
+    private static int getLastCharPositionInLine(Token token)
+    {
+        return token.getCharPositionInLine() + getLength(token);
+    }
+
+    /**
+     * Return the token length.
+     *
+     * @param token the token
+     * @return the token length
+     */
+    private static int getLength(Token token)
+    {
+        return token.getText().length();
+    }
+
+    /**
+     * Inserts a character at a given position within a <code>String</code>.
+     *
+     * @param s the <code>String</code> in which the character must be inserted
+     * @param index the position where the character must be inserted
+     * @param c the character to insert
+     * @return the modified <code>String</code>
+     */
+    private static String insertChar(String s, int index, char c)
+    {
+        return new StringBuilder().append(s.substring(0, index))
+                .append(c)
+                .append(s.substring(index))
+                .toString();
+    }
+
+    /**
+     * Returns the index of the line number on which this token was matched; index=0..n-1
+     *
+     * @param token the token
+     * @return the index of the line number on which this token was matched; index=0..n-1
+     */
+    private static int lineIndex(Token token)
+    {
+        return token.getLine() - 1;
+    }
+
+    /**
+     * Returns the index of the last token which is part of the snippet.
+     *
+     * @param index the index of the token causing the error
+     * @param size the total number of tokens
+     * @return the index of the last token which is part of the snippet.
+     */
+    private static int getSnippetLastTokenIndex(int index, int size)
+    {
+        return Math.min(size - 1, index + LAST_TOKEN_OFFSET);
+    }
+
+    /**
+     * Returns the index of the first token which is part of the snippet.
+     *
+     * @param index the index of the token causing the error
+     * @return the index of the first token which is part of the snippet.
+     */
+    private static int getSnippetFirstTokenIndex(int index)
+    {
+        return Math.max(0, index - FIRST_TOKEN_OFFSET);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8a2d8a13/src/java/org/apache/cassandra/cql3/ErrorListener.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ErrorListener.java b/src/java/org/apache/cassandra/cql3/ErrorListener.java
new file mode 100644
index 0000000..0bf891a
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/ErrorListener.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import org.antlr.runtime.BaseRecognizer;
+import org.antlr.runtime.RecognitionException;
+
+/**
+ * Listener used to collect the syntax errors emitted by the Lexer and Parser.
+ */
+public interface ErrorListener
+{
+    /**
+     * Invoked when a syntax error occurs.
+     *
+     * @param recognizer the parser or lexer that emitted the error
+     * @param tokenNames the token names
+     * @param e the exception
+     */
+    void syntaxError(BaseRecognizer recognizer, String[] tokenNames, RecognitionException e);
+
+    /**
+     * Invoked when a syntax error with a specified message occurs.
+     *
+     * @param recognizer the parser or lexer that emitted the error
+     * @param errorMsg the error message
+     */
+    void syntaxError(BaseRecognizer recognizer, String errorMsg);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8a2d8a13/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index 86362f7..4994342 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -442,18 +442,21 @@ public class QueryProcessor implements QueryHandler
         try
         {
             // Lexer and parser
+            ErrorCollector errorCollector = new ErrorCollector(queryStr);
             CharStream stream = new ANTLRStringStream(queryStr);
             CqlLexer lexer = new CqlLexer(stream);
+            lexer.addErrorListener(errorCollector);
+
             TokenStream tokenStream = new CommonTokenStream(lexer);
             CqlParser parser = new CqlParser(tokenStream);
+            parser.addErrorListener(errorCollector);
 
             // Parse the query string to a statement instance
             ParsedStatement statement = parser.query();
 
-            // The lexer and parser queue up any errors they may have encountered
-            // along the way, if necessary, we turn them into exceptions here.
-            lexer.throwLastRecognitionError();
-            parser.throwLastRecognitionError();
+            // The errorCollector has queue up any errors that the lexer and parser may have encountered
+            // along the way, if necessary, we turn the last error into exceptions here.
+            errorCollector.throwLastSyntaxError();
 
             return statement;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8a2d8a13/test/unit/org/apache/cassandra/cql3/CqlParserTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CqlParserTest.java b/test/unit/org/apache/cassandra/cql3/CqlParserTest.java
new file mode 100644
index 0000000..d122eb5
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/CqlParserTest.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import org.junit.Test;
+
+import org.antlr.runtime.ANTLRStringStream;
+import org.antlr.runtime.BaseRecognizer;
+import org.antlr.runtime.CharStream;
+import org.antlr.runtime.CommonTokenStream;
+import org.antlr.runtime.RecognitionException;
+import org.antlr.runtime.TokenStream;
+
+import static org.junit.Assert.*;
+
+public class CqlParserTest
+{
+    @Test
+    public void testAddErrorListener() throws Exception
+    {
+        SyntaxErrorCounter firstCounter = new SyntaxErrorCounter();
+        SyntaxErrorCounter secondCounter = new SyntaxErrorCounter();
+
+        CharStream stream = new ANTLRStringStream("SELECT * FORM test;");
+        CqlLexer lexer = new CqlLexer(stream);
+
+        TokenStream tokenStream = new CommonTokenStream(lexer);
+        CqlParser parser = new CqlParser(tokenStream);
+        parser.addErrorListener(firstCounter);
+        parser.addErrorListener(secondCounter);
+
+        parser.query();
+
+        assertEquals(1, firstCounter.count);
+        assertEquals(1, secondCounter.count);
+    }
+
+    @Test
+    public void testRemoveErrorListener() throws Exception
+    {
+        SyntaxErrorCounter firstCounter = new SyntaxErrorCounter();
+        SyntaxErrorCounter secondCounter = new SyntaxErrorCounter();
+
+        CharStream stream = new ANTLRStringStream("SELECT * FORM test;");
+        CqlLexer lexer = new CqlLexer(stream);
+
+        TokenStream tokenStream = new CommonTokenStream(lexer);
+        CqlParser parser = new CqlParser(tokenStream);
+        parser.addErrorListener(firstCounter);
+        parser.addErrorListener(secondCounter);
+        parser.removeErrorListener(secondCounter);
+
+        parser.query();
+
+        assertEquals(1, firstCounter.count);
+        assertEquals(0, secondCounter.count);
+    }
+
+    private static final class SyntaxErrorCounter implements ErrorListener
+    {
+        private int count;
+
+        @Override
+        public void syntaxError(BaseRecognizer recognizer, String[] tokenNames, RecognitionException e)
+        {
+            count++;
+        }
+
+        @Override
+        public void syntaxError(BaseRecognizer recognizer, String errorMsg)
+        {
+            count++;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8a2d8a13/test/unit/org/apache/cassandra/cql3/ErrorCollectorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ErrorCollectorTest.java b/test/unit/org/apache/cassandra/cql3/ErrorCollectorTest.java
new file mode 100644
index 0000000..4f5db34
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/ErrorCollectorTest.java
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import org.antlr.runtime.CharStream;
+import org.antlr.runtime.Token;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class ErrorCollectorTest
+{
+    @Test
+    public void testAppendSnippetWithEmptyQuery()
+    {
+        String query = ";";
+
+        ErrorCollector collector = new ErrorCollector(query);
+
+        StringBuilder builder = new StringBuilder();
+
+        Token from = new MockToken(1, 0, ";");
+        Token to = new MockToken(1, 0, ";");
+        Token offending = new MockToken(1, 0, ";");
+
+        collector.appendSnippet(builder, from, to, offending);
+
+        String expected = " ([;])";
+
+        assertEquals(expected, builder.toString());
+    }
+
+    @Test
+    public void testAppendSnippetWithOneLines()
+    {
+        String query = "select * from users where user_name = ''test'';";
+
+        ErrorCollector collector = new ErrorCollector(query);
+
+        StringBuilder builder = new StringBuilder();
+
+        Token from = new MockToken(1, 25, " ");
+        Token to = new MockToken(1, 46, ";");
+        Token offending = new MockToken(1, 40, "test");
+
+        collector.appendSnippet(builder, from, to, offending);
+
+        String expected = " (... user_name = ''[test]'';)";
+
+        assertEquals(expected, builder.toString());
+    }
+
+    @Test
+    public void testAppendSnippetOnSecondLine()
+    {
+        String query = "select * from users\n" +
+                "where user_name = ''test'';";
+
+        ErrorCollector collector = new ErrorCollector(query);
+
+        StringBuilder builder = new StringBuilder();
+
+        Token from = new MockToken(2, 5, " ");
+        Token to = new MockToken(2, 26, ";");
+        Token offending = new MockToken(2, 20, "test");
+
+        collector.appendSnippet(builder, from, to, offending);
+
+        String expected = " (... user_name = ''[test]'';)";
+
+        assertEquals(expected, builder.toString());
+    }
+
+    @Test
+    public void testAppendSnippetWithSnippetOverTwoLines()
+    {
+        String query = "select * from users where user_name \n" +
+                "= ''test'';";
+
+        ErrorCollector collector = new ErrorCollector(query);
+
+        StringBuilder builder = new StringBuilder();
+
+        Token from = new MockToken(1, 20, "where");
+        Token to = new MockToken(2, 9, "'");
+        Token offending = new MockToken(2, 4, "test");
+
+        collector.appendSnippet(builder, from, to, offending);
+
+        String expected = " (...where user_name = ''[test]''...)";
+
+        assertEquals(expected, builder.toString());
+    }
+
+    private final static class MockToken implements Token
+    {
+        /**
+         * The line number on which this token was matched; line=1..n
+         */
+        private int line;
+
+        /**
+         * The index of the first character relative to the beginning of the line 0..n-1
+         */
+        private int charPositionInLine;
+
+        /**
+         * The text of the token
+         */
+        private String text;
+
+        public MockToken(int line, int charPositionInLine, String text)
+        {
+            this.line = line;
+            this.charPositionInLine = charPositionInLine;
+            this.text = text;
+        }
+
+        @Override
+        public int getChannel()
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public int getCharPositionInLine()
+        {
+            return charPositionInLine;
+        }
+
+        @Override
+        public CharStream getInputStream()
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public int getLine()
+        {
+            return line;
+        }
+
+        @Override
+        public String getText()
+        {
+            return text;
+        }
+
+        @Override
+        public int getTokenIndex()
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public int getType()
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void setChannel(int channel)
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void setCharPositionInLine(int charPositionInLine)
+        {
+            this.charPositionInLine = charPositionInLine;
+        }
+
+        @Override
+        public void setInputStream(CharStream inputStream)
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void setLine(int line)
+        {
+            this.line = line;
+        }
+
+        @Override
+        public void setText(String text)
+        {
+            this.text = text;
+        }
+
+        @Override
+        public void setTokenIndex(int tokenIndex)
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void setType(int type)
+        {
+            throw new UnsupportedOperationException();
+        }
+    }
+}


[2/2] git commit: Merge branch 'cassandra-2.1' into trunk

Posted by ty...@apache.org.
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 8a5a82b8a87314d1e740d94cd40a190bd2681a42
Parents: 3966935 8a2d8a1
Author: Tyler Hobbs <ty...@datastax.com>
Authored: Tue Jul 15 12:06:51 2014 -0500
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Tue Jul 15 12:06:51 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 src/java/org/apache/cassandra/cql3/Cql.g        |  41 ++--
 .../apache/cassandra/cql3/ErrorCollector.java   | 242 +++++++++++++++++++
 .../apache/cassandra/cql3/ErrorListener.java    |  44 ++++
 .../apache/cassandra/cql3/QueryProcessor.java   |  11 +-
 .../apache/cassandra/cql3/CqlParserTest.java    |  90 +++++++
 .../cassandra/cql3/ErrorCollectorTest.java      | 218 +++++++++++++++++
 7 files changed, 621 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8a5a82b8/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8a5a82b8/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------

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