You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2012/09/05 17:51:59 UTC

[3/6] Finer grained exception hierarchy, and adds error codes

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/db/marshal/SetType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/SetType.java b/src/java/org/apache/cassandra/db/marshal/SetType.java
index 7c71baf..45b6e54 100644
--- a/src/java/org/apache/cassandra/db/marshal/SetType.java
+++ b/src/java/org/apache/cassandra/db/marshal/SetType.java
@@ -21,7 +21,8 @@ import java.nio.ByteBuffer;
 import java.util.*;
 
 import org.apache.cassandra.db.IColumn;
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
@@ -33,7 +34,7 @@ public class SetType<T> extends CollectionType<Set<T>>
 
     public final AbstractType<T> elements;
 
-    public static SetType<?> getInstance(TypeParser parser) throws ConfigurationException
+    public static SetType<?> getInstance(TypeParser parser) throws ConfigurationException, SyntaxException
     {
         List<AbstractType<?>> l = parser.getTypeParameters();
         if (l.size() != 1)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/db/marshal/TypeParser.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/TypeParser.java b/src/java/org/apache/cassandra/db/marshal/TypeParser.java
index 5380638..5813250 100644
--- a/src/java/org/apache/cassandra/db/marshal/TypeParser.java
+++ b/src/java/org/apache/cassandra/db/marshal/TypeParser.java
@@ -29,7 +29,7 @@ import java.util.Map;
 
 import org.apache.commons.lang.StringUtils;
 
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -60,7 +60,7 @@ public class TypeParser
     /**
      * Parse a string containing an type definition.
      */
-    public static AbstractType<?> parse(String str) throws ConfigurationException
+    public static AbstractType<?> parse(String str) throws SyntaxException, ConfigurationException
     {
         if (str == null)
             return BytesType.instance;
@@ -93,7 +93,7 @@ public class TypeParser
         return type;
     }
 
-    public static AbstractType<?> parse(CharSequence compareWith) throws ConfigurationException
+    public static AbstractType<?> parse(CharSequence compareWith) throws SyntaxException, ConfigurationException
     {
         return parse(compareWith == null ? null : compareWith.toString());
     }
@@ -106,7 +106,7 @@ public class TypeParser
     /**
      * Parse an AbstractType from current position of this parser.
      */
-    public AbstractType<?> parse() throws ConfigurationException
+    public AbstractType<?> parse() throws SyntaxException, ConfigurationException
     {
         skipBlank();
         String name = readNextIdentifier();
@@ -118,7 +118,7 @@ public class TypeParser
             return getAbstractType(name);
     }
 
-    public Map<String, String> getKeyValueParameters() throws ConfigurationException
+    public Map<String, String> getKeyValueParameters() throws SyntaxException
     {
         Map<String, String> map = new HashMap<String, String>();
 
@@ -153,10 +153,10 @@ public class TypeParser
             }
             map.put(k, v);
         }
-        throw new ConfigurationException(String.format("Syntax error parsing '%s' at char %d: unexpected end of string", str, idx));
+        throw new SyntaxException(String.format("Syntax error parsing '%s' at char %d: unexpected end of string", str, idx));
     }
 
-    public List<AbstractType<?>> getTypeParameters() throws ConfigurationException
+    public List<AbstractType<?>> getTypeParameters() throws SyntaxException, ConfigurationException
     {
         List<AbstractType<?>> list = new ArrayList<AbstractType<?>>();
 
@@ -180,17 +180,17 @@ public class TypeParser
             {
                 list.add(parse());
             }
-            catch (ConfigurationException e)
+            catch (SyntaxException e)
             {
-                ConfigurationException ex = new ConfigurationException(String.format("Exception while parsing '%s' around char %d", str, idx));
+                SyntaxException ex = new SyntaxException(String.format("Exception while parsing '%s' around char %d", str, idx));
                 ex.initCause(e);
                 throw ex;
             }
         }
-        throw new ConfigurationException(String.format("Syntax error parsing '%s' at char %d: unexpected end of string", str, idx));
+        throw new SyntaxException(String.format("Syntax error parsing '%s' at char %d: unexpected end of string", str, idx));
     }
 
-    public Map<Byte, AbstractType<?>> getAliasParameters() throws ConfigurationException
+    public Map<Byte, AbstractType<?>> getAliasParameters() throws SyntaxException, ConfigurationException
     {
         Map<Byte, AbstractType<?>> map = new HashMap<Byte, AbstractType<?>>();
 
@@ -228,17 +228,17 @@ public class TypeParser
             {
                 map.put((byte)aliasChar, parse());
             }
-            catch (ConfigurationException e)
+            catch (SyntaxException e)
             {
-                ConfigurationException ex = new ConfigurationException(String.format("Exception while parsing '%s' around char %d", str, idx));
+                SyntaxException ex = new SyntaxException(String.format("Exception while parsing '%s' around char %d", str, idx));
                 ex.initCause(e);
                 throw ex;
             }
         }
-        throw new ConfigurationException(String.format("Syntax error parsing '%s' at char %d: unexpected end of string", str, idx));
+        throw new SyntaxException(String.format("Syntax error parsing '%s' at char %d: unexpected end of string", str, idx));
     }
 
-    public Map<ByteBuffer, CollectionType> getCollectionsParameters() throws ConfigurationException
+    public Map<ByteBuffer, CollectionType> getCollectionsParameters() throws SyntaxException, ConfigurationException
     {
         Map<ByteBuffer, CollectionType> map = new HashMap<ByteBuffer, CollectionType>();
 
@@ -279,17 +279,17 @@ public class TypeParser
             {
                 AbstractType<?> type = parse();
                 if (!(type instanceof CollectionType))
-                    throw new ConfigurationException(type.toString() + " is not a collection type");
+                    throw new SyntaxException(type.toString() + " is not a collection type");
                 map.put(bb, (CollectionType)type);
             }
-            catch (ConfigurationException e)
+            catch (SyntaxException e)
             {
-                ConfigurationException ex = new ConfigurationException(String.format("Exception while parsing '%s' around char %d", str, idx));
+                SyntaxException ex = new SyntaxException(String.format("Exception while parsing '%s' around char %d", str, idx));
                 ex.initCause(e);
                 throw ex;
             }
         }
-        throw new ConfigurationException(String.format("Syntax error parsing '%s' at char %d: unexpected end of string", str, idx));
+        throw new SyntaxException(String.format("Syntax error parsing '%s' at char %d: unexpected end of string", str, idx));
     }
 
     private static AbstractType<?> getAbstractType(String compareWith) throws ConfigurationException
@@ -313,7 +313,7 @@ public class TypeParser
         }
     }
 
-    private static AbstractType<?> getAbstractType(String compareWith, TypeParser parser) throws ConfigurationException
+    private static AbstractType<?> getAbstractType(String compareWith, TypeParser parser) throws SyntaxException, ConfigurationException
     {
         String className = compareWith.contains(".") ? compareWith : "org.apache.cassandra.db.marshal." + compareWith;
         Class<? extends AbstractType<?>> typeClass = FBUtilities.<AbstractType<?>>classForName(className, "abstract-type");
@@ -382,9 +382,9 @@ public class TypeParser
         }
     }
 
-    private void throwSyntaxError(String msg) throws ConfigurationException
+    private void throwSyntaxError(String msg) throws SyntaxException
     {
-        throw new ConfigurationException(String.format("Syntax error parsing '%s' at char %d: %s", str, idx, msg));
+        throw new SyntaxException(String.format("Syntax error parsing '%s' at char %d: %s", str, idx, msg));
     }
 
     private boolean isEOS()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java b/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java
index 36a5173..cff4754 100644
--- a/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java
@@ -26,6 +26,7 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.commons.lang.ArrayUtils;
 
 import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Hex;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/dht/BootStrapper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/BootStrapper.java b/src/java/org/apache/cassandra/dht/BootStrapper.java
index 1942be8..d6c0fc2 100644
--- a/src/java/org/apache/cassandra/dht/BootStrapper.java
+++ b/src/java/org/apache/cassandra/dht/BootStrapper.java
@@ -28,9 +28,8 @@ import java.util.concurrent.locks.Condition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.gms.*;
 
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Table;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
index fb8d721..3a3972b 100644
--- a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
+++ b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
@@ -24,8 +24,8 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.cassandra.config.ConfigurationException;
 import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.MurmurHash;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
index f0a83f3..25f26f4 100644
--- a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
@@ -24,6 +24,7 @@ import java.util.*;
 
 import org.apache.cassandra.config.*;
 import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.VersionedValue;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/dht/RandomPartitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/RandomPartitioner.java b/src/java/org/apache/cassandra/dht/RandomPartitioner.java
index cf3855b..da11fd1 100644
--- a/src/java/org/apache/cassandra/dht/RandomPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/RandomPartitioner.java
@@ -23,7 +23,7 @@ import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
 import java.util.*;
 
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/dht/Token.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/Token.java b/src/java/org/apache/cassandra/dht/Token.java
index 2aeafbd..d392972 100644
--- a/src/java/org/apache/cassandra/dht/Token.java
+++ b/src/java/org/apache/cassandra/dht/Token.java
@@ -23,7 +23,7 @@ import java.io.IOException;
 import java.io.Serializable;
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.RowPosition;
 import org.apache.cassandra.io.ISerializer;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/exceptions/AlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/AlreadyExistsException.java b/src/java/org/apache/cassandra/exceptions/AlreadyExistsException.java
new file mode 100644
index 0000000..f023a07
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/AlreadyExistsException.java
@@ -0,0 +1,42 @@
+/*
+ * 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.exceptions;
+
+public class AlreadyExistsException extends ConfigurationException
+{
+    public final String ksName;
+    public final String cfName;
+
+    private AlreadyExistsException(String ksName, String cfName, String msg)
+    {
+        super(ExceptionCode.ALREADY_EXISTS, msg);
+        this.ksName = ksName;
+        this.cfName = cfName;
+    }
+
+    public AlreadyExistsException(String ksName, String cfName)
+    {
+        this(ksName, cfName, String.format("Cannot add already existing column family \"%s\" to keyspace \"%s\"", ksName, cfName));
+    }
+
+    public AlreadyExistsException(String ksName)
+    {
+        this(ksName, "", String.format("Cannot add existing keyspace \"%s\"", ksName));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/exceptions/CassandraException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/CassandraException.java b/src/java/org/apache/cassandra/exceptions/CassandraException.java
new file mode 100644
index 0000000..00790d6
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/CassandraException.java
@@ -0,0 +1,42 @@
+/*
+ * 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.exceptions;
+
+import java.nio.ByteBuffer;
+
+public abstract class CassandraException extends Exception implements TransportException
+{
+    private final ExceptionCode code;
+
+    protected CassandraException(ExceptionCode code, String msg)
+    {
+        super(msg);
+        this.code = code;
+    }
+
+    protected CassandraException(ExceptionCode code, String msg, Throwable cause)
+    {
+        super(msg, cause);
+        this.code = code;
+    }
+
+    public ExceptionCode code()
+    {
+        return code;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/exceptions/ConfigurationException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/ConfigurationException.java b/src/java/org/apache/cassandra/exceptions/ConfigurationException.java
new file mode 100644
index 0000000..8f0bb1a
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/ConfigurationException.java
@@ -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.exceptions;
+
+public class ConfigurationException extends RequestValidationException
+{
+    public ConfigurationException(String msg)
+    {
+        super(ExceptionCode.CONFIG_ERROR, msg);
+    }
+
+    public ConfigurationException(String msg, Throwable e)
+    {
+        super(ExceptionCode.CONFIG_ERROR, msg, e);
+    }
+
+    protected ConfigurationException(ExceptionCode code, String msg)
+    {
+        super(code, msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/exceptions/ExceptionCode.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/ExceptionCode.java b/src/java/org/apache/cassandra/exceptions/ExceptionCode.java
new file mode 100644
index 0000000..13fcc6a
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/ExceptionCode.java
@@ -0,0 +1,68 @@
+/*
+ * 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.exceptions;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.cassandra.transport.ProtocolException;
+
+/**
+ * Exceptions code, as defined by the binary protocol.
+ */
+public enum ExceptionCode
+{
+    SERVER_ERROR    (0x0000),
+    PROTOCOL_ERROR  (0x000A),
+
+    // 1xx: problem during request execution
+    UNAVAILABLE     (0x1000),
+    OVERLOADED      (0x1001),
+    IS_BOOTSTRAPPING(0x1002),
+    TRUNCATE_ERROR  (0x1003),
+    WRITE_TIMEOUT   (0x1100),
+    READ_TIMEOUT    (0x1200),
+
+    // 2xx: problem validating the request
+    SYNTAX_ERROR    (0x2000),
+    UNAUTHORIZED    (0x2100),
+    INVALID         (0x2200),
+    CONFIG_ERROR    (0x2300),
+    ALREADY_EXISTS  (0x2400);
+
+    public final int value;
+    private static final Map<Integer, ExceptionCode> valueToCode = new HashMap<Integer, ExceptionCode>(ExceptionCode.values().length);
+    static
+    {
+        for (ExceptionCode code : ExceptionCode.values())
+            valueToCode.put(code.value, code);
+    }
+
+    private ExceptionCode(int value)
+    {
+        this.value = value;
+    }
+
+    public static ExceptionCode fromValue(int value)
+    {
+        ExceptionCode code = valueToCode.get(value);
+        if (code == null)
+            throw new ProtocolException(String.format("Unknown error code %d", value));
+        return code;
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/exceptions/IsBootstrappingException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/IsBootstrappingException.java b/src/java/org/apache/cassandra/exceptions/IsBootstrappingException.java
new file mode 100644
index 0000000..b25a318
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/IsBootstrappingException.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.exceptions;
+
+public class IsBootstrappingException extends RequestExecutionException
+{
+    public IsBootstrappingException()
+    {
+        super(ExceptionCode.IS_BOOTSTRAPPING, "Cannot read from a bootstrapping node");
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/exceptions/ReadTimeoutException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/ReadTimeoutException.java b/src/java/org/apache/cassandra/exceptions/ReadTimeoutException.java
new file mode 100644
index 0000000..3af638d
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/ReadTimeoutException.java
@@ -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.exceptions;
+
+import java.net.InetAddress;
+import java.util.Set;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class ReadTimeoutException extends RequestTimeoutException
+{
+    public final boolean dataPresent;
+
+    public ReadTimeoutException(ConsistencyLevel consistency, int received, int blockFor, boolean dataPresent)
+    {
+        super(ExceptionCode.READ_TIMEOUT, consistency, received, blockFor);
+        this.dataPresent = dataPresent;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/exceptions/RequestExecutionException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/RequestExecutionException.java b/src/java/org/apache/cassandra/exceptions/RequestExecutionException.java
new file mode 100644
index 0000000..8d9839f
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/RequestExecutionException.java
@@ -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.exceptions;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+
+public abstract class RequestExecutionException extends CassandraException
+{
+    protected RequestExecutionException(ExceptionCode code, String msg)
+    {
+        super(code, msg);
+    }
+
+    protected RequestExecutionException(ExceptionCode code, String msg, Throwable e)
+    {
+        super(code, msg, e);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/exceptions/RequestTimeoutException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/RequestTimeoutException.java b/src/java/org/apache/cassandra/exceptions/RequestTimeoutException.java
new file mode 100644
index 0000000..b6cfb86
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/RequestTimeoutException.java
@@ -0,0 +1,40 @@
+/*
+ * 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.exceptions;
+
+import java.net.InetAddress;
+import java.util.Set;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class RequestTimeoutException extends RequestExecutionException
+{
+    public final ConsistencyLevel consistency;
+    public final int received;
+    public final int blockFor;
+
+    protected RequestTimeoutException(ExceptionCode code, ConsistencyLevel consistency, int received, int blockFor)
+    {
+        super(code, String.format("Operation timed out - received only %d responses.", received));
+        this.consistency = consistency;
+        this.received = received;
+        this.blockFor = blockFor;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/exceptions/RequestValidationException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/RequestValidationException.java b/src/java/org/apache/cassandra/exceptions/RequestValidationException.java
new file mode 100644
index 0000000..a168274
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/RequestValidationException.java
@@ -0,0 +1,31 @@
+/*
+ * 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.exceptions;
+
+public abstract class RequestValidationException extends CassandraException
+{
+    protected RequestValidationException(ExceptionCode code, String msg)
+    {
+        super(code, msg);
+    }
+
+    protected RequestValidationException(ExceptionCode code, String msg, Throwable e)
+    {
+        super(code, msg, e);
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/exceptions/TransportException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/TransportException.java b/src/java/org/apache/cassandra/exceptions/TransportException.java
new file mode 100644
index 0000000..940f723
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/TransportException.java
@@ -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.exceptions;
+
+import java.nio.ByteBuffer;
+
+public interface TransportException
+{
+    /**
+     * The exception code.
+     */
+    public ExceptionCode code();
+
+    /**
+     * The exception message.
+     */
+    public String getMessage();
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/exceptions/TruncateException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/TruncateException.java b/src/java/org/apache/cassandra/exceptions/TruncateException.java
new file mode 100644
index 0000000..6d6e027
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/TruncateException.java
@@ -0,0 +1,31 @@
+/*
+ * 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.exceptions;
+
+public class TruncateException extends RequestExecutionException
+{
+    public TruncateException(Throwable e)
+    {
+        super(ExceptionCode.TRUNCATE_ERROR, "Error during truncate: " + e.getMessage(), e);
+    }
+
+    public TruncateException(String msg)
+    {
+        super(ExceptionCode.TRUNCATE_ERROR, msg);
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/exceptions/UnavailableException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/UnavailableException.java b/src/java/org/apache/cassandra/exceptions/UnavailableException.java
new file mode 100644
index 0000000..ceb87d8
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/UnavailableException.java
@@ -0,0 +1,40 @@
+/*
+ * 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.exceptions;
+
+import java.net.InetAddress;
+import java.util.Collection;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class UnavailableException extends RequestExecutionException
+{
+    public final ConsistencyLevel consistency;
+    public final int required;
+    public final int alive;
+
+    public UnavailableException(ConsistencyLevel consistency, int required, int alive)
+    {
+        super(ExceptionCode.UNAVAILABLE, "Cannot achieve consistency level " + consistency);
+        this.consistency = consistency;
+        this.required = required;
+        this.alive = alive;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/exceptions/WriteTimeoutException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/WriteTimeoutException.java b/src/java/org/apache/cassandra/exceptions/WriteTimeoutException.java
new file mode 100644
index 0000000..607215b
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/WriteTimeoutException.java
@@ -0,0 +1,32 @@
+/*
+ * 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.exceptions;
+
+import java.net.InetAddress;
+import java.util.Set;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+
+public class WriteTimeoutException extends RequestTimeoutException
+{
+    public WriteTimeoutException(ConsistencyLevel consistency, int received, int blockFor)
+    {
+        super(ExceptionCode.WRITE_TIMEOUT, consistency, received, blockFor);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
index db284ec..3385b35 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
@@ -31,7 +31,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.IColumn;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.TypeParser;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ConfigHelper.java b/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
index 1374162..0c390cc 100644
--- a/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
+++ b/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
@@ -30,7 +30,7 @@ import org.apache.cassandra.io.compress.CompressionParameters;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.thrift.*;
 import org.apache.cassandra.utils.FBUtilities;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
index 33bddfe..c0c2997 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
@@ -22,11 +22,12 @@ import java.math.BigInteger;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.config.ConfigurationException;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.IntegerType;
 import org.apache.cassandra.db.marshal.TypeParser;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.thrift.*;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Hex;
@@ -402,6 +403,10 @@ public class CassandraStorage extends LoadFunc implements StoreFuncInterface, Lo
                 {
                     throw new IOException(e);
                 }
+                catch (SyntaxException e)
+                {
+                    throw new IOException(e);
+                }
             }
         }
         return validators;
@@ -420,6 +425,10 @@ public class CassandraStorage extends LoadFunc implements StoreFuncInterface, Lo
         {
             throw new IOException(e);
         }
+        catch (SyntaxException e)
+        {
+            throw new IOException(e);
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
index a68de8a..5587efc 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
@@ -22,7 +22,7 @@ import java.util.*;
 
 import com.google.common.primitives.Longs;
 
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.FSWriteError;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/io/compress/CompressionParameters.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/CompressionParameters.java b/src/java/org/apache/cassandra/io/compress/CompressionParameters.java
index 4ee7bef..25fc742 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressionParameters.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressionParameters.java
@@ -30,7 +30,7 @@ import java.util.Set;
 import org.apache.commons.lang.builder.EqualsBuilder;
 import org.apache.commons.lang.builder.HashCodeBuilder;
 
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
index 49d3219..13e2b99 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
@@ -28,7 +28,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.cassandra.config.Config;
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
index 66fb849..3b81ab8 100644
--- a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
+++ b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
@@ -26,7 +26,7 @@ import com.google.common.collect.Multimap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.RingPosition;
 import org.apache.cassandra.dht.Token;
@@ -34,7 +34,7 @@ import org.apache.cassandra.service.DatacenterSyncWriteResponseHandler;
 import org.apache.cassandra.service.DatacenterWriteResponseHandler;
 import org.apache.cassandra.service.IWriteResponseHandler;
 import org.apache.cassandra.service.WriteResponseHandler;
-import org.apache.cassandra.thrift.ConsistencyLevel;
+import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.utils.FBUtilities;
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java b/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
index 4c6ab32..e29637f 100644
--- a/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
+++ b/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.EndpointState;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/locator/Ec2Snitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/Ec2Snitch.java b/src/java/org/apache/cassandra/locator/Ec2Snitch.java
index 729121d..5485977 100644
--- a/src/java/org/apache/cassandra/locator/Ec2Snitch.java
+++ b/src/java/org/apache/cassandra/locator/Ec2Snitch.java
@@ -28,7 +28,7 @@ import com.google.common.base.Charsets;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.EndpointState;
 import org.apache.cassandra.gms.Gossiper;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java b/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java
index 9a22544..adc8b24 100644
--- a/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java
+++ b/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java
@@ -18,7 +18,7 @@
 
 package org.apache.cassandra.locator;
 
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.EndpointState;
 import org.apache.cassandra.gms.Gossiper;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/locator/LocalStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/LocalStrategy.java b/src/java/org/apache/cassandra/locator/LocalStrategy.java
index 09318de..17ad181 100644
--- a/src/java/org/apache/cassandra/locator/LocalStrategy.java
+++ b/src/java/org/apache/cassandra/locator/LocalStrategy.java
@@ -23,7 +23,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.dht.RingPosition;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.utils.FBUtilities;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
index 7086864..8b251d7 100644
--- a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
@@ -24,7 +24,7 @@ import java.util.Map.Entry;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.locator.TokenMetadata.Topology;
 import org.apache.cassandra.utils.FBUtilities;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java b/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
index 4d8db9d..f07328b 100644
--- a/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
@@ -23,7 +23,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.dht.Token;
 
 /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java b/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java
index 675297a..1c25ca8 100644
--- a/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java
+++ b/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java
@@ -27,7 +27,7 @@ import java.util.Properties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/locator/SimpleStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/SimpleStrategy.java b/src/java/org/apache/cassandra/locator/SimpleStrategy.java
index 50d470f..7f27334 100644
--- a/src/java/org/apache/cassandra/locator/SimpleStrategy.java
+++ b/src/java/org/apache/cassandra/locator/SimpleStrategy.java
@@ -24,7 +24,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.dht.Token;
 
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java
index c4ceae2..1a22273 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -43,7 +43,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.EncryptionOptions;
 import org.apache.cassandra.db.*;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
index 4c1a83b..328e1ea 100644
--- a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
@@ -22,10 +22,10 @@ import java.util.Collection;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.exceptions.UnavailableException;
+import org.apache.cassandra.exceptions.WriteTimeoutException;
 import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.thrift.ConsistencyLevel;
-import org.apache.cassandra.thrift.TimedOutException;
-import org.apache.cassandra.thrift.UnavailableException;
+import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.utils.SimpleCondition;
 
 public abstract class AbstractWriteResponseHandler implements IWriteResponseHandler
@@ -42,7 +42,7 @@ public abstract class AbstractWriteResponseHandler implements IWriteResponseHand
         this.writeEndpoints = writeEndpoints;
     }
 
-    public void get() throws TimedOutException
+    public void get() throws WriteTimeoutException
     {
         long timeout = DatabaseDescriptor.getWriteRpcTimeout() - (System.currentTimeMillis() - startTime);
 
@@ -57,13 +57,13 @@ public abstract class AbstractWriteResponseHandler implements IWriteResponseHand
         }
 
         if (!success)
-        {
-            throw new TimedOutException().setAcknowledged_by(ackCount());
-        }
+            throw new WriteTimeoutException(consistencyLevel, ackCount(), blockFor());
     }
 
     protected abstract int ackCount();
 
+    protected abstract int blockFor();
+
     /** null message means "response from local write" */
     public abstract void response(MessageIn msg);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index 4e64fa3..773aa47 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -32,7 +32,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.*;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/service/ClientState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java
index a04f3f1..b85b359 100644
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@ -30,8 +30,9 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql.CQLStatement;
 import org.apache.cassandra.db.Table;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.UnauthorizedException;
 import org.apache.cassandra.thrift.AuthenticationException;
-import org.apache.cassandra.thrift.InvalidRequestException;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.SemanticVersion;
 
@@ -185,7 +186,7 @@ public class ClientState
     /**
      * Confirms that the client thread has the given Permission for the Keyspace list.
      */
-    public void hasKeyspaceSchemaAccess(Permission perm) throws InvalidRequestException
+    public void hasKeyspaceSchemaAccess(Permission perm) throws UnauthorizedException, InvalidRequestException
     {
         validateLogin();
 
@@ -195,7 +196,7 @@ public class ClientState
         hasAccess(user, perms, perm, resource);
     }
 
-    public void hasColumnFamilySchemaAccess(Permission perm) throws InvalidRequestException
+    public void hasColumnFamilySchemaAccess(Permission perm) throws UnauthorizedException, InvalidRequestException
     {
         hasColumnFamilySchemaAccess(keyspace, perm);
     }
@@ -204,14 +205,14 @@ public class ClientState
      * Confirms that the client thread has the given Permission for the ColumnFamily list of
      * the provided keyspace.
      */
-    public void hasColumnFamilySchemaAccess(String keyspace, Permission perm) throws InvalidRequestException
+    public void hasColumnFamilySchemaAccess(String keyspace, Permission perm) throws UnauthorizedException, InvalidRequestException
     {
         validateLogin();
         validateKeyspace(keyspace);
 
         // hardcode disallowing messing with system keyspace
         if (keyspace.equalsIgnoreCase(Table.SYSTEM_KS) && perm == Permission.WRITE)
-            throw new InvalidRequestException("system keyspace is not user-modifiable");
+            throw new UnauthorizedException("system keyspace is not user-modifiable");
 
         resourceClear();
         resource.add(keyspace);
@@ -224,12 +225,12 @@ public class ClientState
      * Confirms that the client thread has the given Permission in the context of the given
      * ColumnFamily and the current keyspace.
      */
-    public void hasColumnFamilyAccess(String columnFamily, Permission perm) throws InvalidRequestException
+    public void hasColumnFamilyAccess(String columnFamily, Permission perm) throws UnauthorizedException, InvalidRequestException
     {
         hasColumnFamilyAccess(keyspace, columnFamily, perm);
     }
 
-    public void hasColumnFamilyAccess(String keyspace, String columnFamily, Permission perm) throws InvalidRequestException
+    public void hasColumnFamilyAccess(String keyspace, String columnFamily, Permission perm) throws UnauthorizedException, InvalidRequestException
     {
         validateLogin();
         validateKeyspace(keyspace);
@@ -259,14 +260,14 @@ public class ClientState
             throw new InvalidRequestException("You have not set a keyspace for this session");
     }
 
-    private static void hasAccess(AuthenticatedUser user, Set<Permission> perms, Permission perm, List<Object> resource) throws InvalidRequestException
+    private static void hasAccess(AuthenticatedUser user, Set<Permission> perms, Permission perm, List<Object> resource) throws UnauthorizedException
     {
         if (perms.contains(perm))
             return;
-        throw new InvalidRequestException(String.format("%s does not have permission %s for %s",
-                                                        user,
-                                                        perm,
-                                                        Resources.toString(resource)));
+        throw new UnauthorizedException(String.format("%s does not have permission %s for %s",
+                                                      user,
+                                                      perm,
+                                                       Resources.toString(resource)));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/service/DatacenterReadCallback.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/DatacenterReadCallback.java b/src/java/org/apache/cassandra/service/DatacenterReadCallback.java
index 76d6f01..84e0c59 100644
--- a/src/java/org/apache/cassandra/service/DatacenterReadCallback.java
+++ b/src/java/org/apache/cassandra/service/DatacenterReadCallback.java
@@ -22,12 +22,12 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 
+import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.ReadResponse;
 import org.apache.cassandra.db.Table;
+import org.apache.cassandra.exceptions.UnavailableException;
 import org.apache.cassandra.locator.NetworkTopologyStrategy;
 import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.thrift.ConsistencyLevel;
-import org.apache.cassandra.thrift.UnavailableException;
 import org.apache.thrift.protocol.TMessage;
 
 /**
@@ -67,13 +67,6 @@ public class DatacenterReadCallback<TMessage, TResolved> extends ReadCallback<TM
     }
 
     @Override
-    public int determineBlockFor(ConsistencyLevel consistency_level, String table)
-    {
-        NetworkTopologyStrategy stategy = (NetworkTopologyStrategy) Table.open(table).getReplicationStrategy();
-        return (stategy.getReplicationFactor(localdc) / 2) + 1;
-    }
-
-    @Override
     public void assureSufficientLiveNodes() throws UnavailableException
     {
         int localEndpoints = 0;
@@ -97,7 +90,7 @@ public class DatacenterReadCallback<TMessage, TResolved> extends ReadCallback<TM
                 logger.debug(builder.toString());
             }
 
-            throw new UnavailableException();
+            throw new UnavailableException(consistencyLevel, blockfor, localEndpoints);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java b/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
index 0dc2ac5..a33ce60 100644
--- a/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
@@ -25,12 +25,12 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Table;
+import org.apache.cassandra.exceptions.UnavailableException;
 import org.apache.cassandra.gms.FailureDetector;
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.locator.NetworkTopologyStrategy;
 import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.thrift.ConsistencyLevel;
-import org.apache.cassandra.thrift.UnavailableException;
+import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
@@ -46,6 +46,7 @@ public class DatacenterSyncWriteResponseHandler extends AbstractWriteResponseHan
         localdc = snitch.getDatacenter(FBUtilities.getBroadcastAddress());
     }
 
+    private final String table;
     private final NetworkTopologyStrategy strategy;
     private final HashMap<String, AtomicInteger> responses = new HashMap<String, AtomicInteger>();
 
@@ -55,6 +56,7 @@ public class DatacenterSyncWriteResponseHandler extends AbstractWriteResponseHan
         super(writeEndpoints, consistencyLevel);
         assert consistencyLevel == ConsistencyLevel.EACH_QUORUM;
 
+        this.table = table;
         strategy = (NetworkTopologyStrategy) Table.open(table).getReplicationStrategy();
 
         for (String dc : strategy.getDatacenters())
@@ -87,6 +89,11 @@ public class DatacenterSyncWriteResponseHandler extends AbstractWriteResponseHan
         condition.signal();
     }
 
+    protected int blockFor()
+    {
+        return consistencyLevel.blockFor(table);
+    }
+
     protected int ackCount()
     {
         int n = 0;
@@ -119,7 +126,7 @@ public class DatacenterSyncWriteResponseHandler extends AbstractWriteResponseHan
         for (String dc: strategy.getDatacenters())
         {
             if (dcEndpoints.get(dc).get() < responses.get(dc).get())
-                throw new UnavailableException();
+                throw new UnavailableException(consistencyLevel, responses.get(dc).get(), dcEndpoints.get(dc).get());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java b/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
index 7592ad2..6644b1a 100644
--- a/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
@@ -22,12 +22,12 @@ import java.util.Collection;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Table;
+import org.apache.cassandra.exceptions.UnavailableException;
 import org.apache.cassandra.gms.FailureDetector;
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.locator.NetworkTopologyStrategy;
 import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.thrift.ConsistencyLevel;
-import org.apache.cassandra.thrift.UnavailableException;
+import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
@@ -55,14 +55,6 @@ public class DatacenterWriteResponseHandler extends WriteResponseHandler
     }
 
     @Override
-    protected int determineBlockFor(String table)
-    {
-        NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) Table.open(table).getReplicationStrategy();
-        return (strategy.getReplicationFactor(localdc) / 2) + 1;
-    }
-
-
-    @Override
     public void response(MessageIn message)
     {
         if (message == null || localdc.equals(snitch.getDatacenter(message.from)))
@@ -84,7 +76,7 @@ public class DatacenterWriteResponseHandler extends WriteResponseHandler
 
         if (liveNodes < responses.get())
         {
-            throw new UnavailableException();
+            throw new UnavailableException(consistencyLevel, responses.get(), liveNodes);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/service/IWriteResponseHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/IWriteResponseHandler.java b/src/java/org/apache/cassandra/service/IWriteResponseHandler.java
index 0127ecd..aa77635 100644
--- a/src/java/org/apache/cassandra/service/IWriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/IWriteResponseHandler.java
@@ -18,11 +18,11 @@
 package org.apache.cassandra.service;
 
 import org.apache.cassandra.net.IAsyncCallback;
-import org.apache.cassandra.thrift.TimedOutException;
-import org.apache.cassandra.thrift.UnavailableException;
+import org.apache.cassandra.exceptions.WriteTimeoutException;
+import org.apache.cassandra.exceptions.UnavailableException;
 
 public interface IWriteResponseHandler extends IAsyncCallback
 {
-    public void get() throws TimedOutException;
+    public void get() throws WriteTimeoutException;
     public void assureSufficientLiveNodes() throws UnavailableException;
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/service/MigrationManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/MigrationManager.java b/src/java/org/apache/cassandra/service/MigrationManager.java
index a31cbbc..f7ba701 100644
--- a/src/java/org/apache/cassandra/service/MigrationManager.java
+++ b/src/java/org/apache/cassandra/service/MigrationManager.java
@@ -34,7 +34,8 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.AlreadyExistsException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.*;
@@ -111,7 +112,7 @@ public class MigrationManager implements IEndpointStateChangeSubscriber
         ksm.validate();
 
         if (Schema.instance.getTableDefinition(ksm.name) != null)
-            throw new ConfigurationException(String.format("Cannot add already existing keyspace '%s'.", ksm.name));
+            throw new AlreadyExistsException(ksm.name);
 
         logger.info(String.format("Create new Keyspace: %s", ksm));
         announce(ksm.toSchema(FBUtilities.timestampMicros()));
@@ -125,7 +126,7 @@ public class MigrationManager implements IEndpointStateChangeSubscriber
         if (ksm == null)
             throw new ConfigurationException(String.format("Cannot add column family '%s' to non existing keyspace '%s'.", cfm.cfName, cfm.ksName));
         else if (ksm.cfMetaData().containsKey(cfm.cfName))
-            throw new ConfigurationException(String.format("Cannot add already existing column family '%s' to keyspace '%s'.", cfm.cfName, cfm.ksName));
+            throw new AlreadyExistsException(cfm.cfName, cfm.ksName);
 
         logger.info(String.format("Create new ColumnFamily: %s", cfm));
         announce(cfm.toSchema(FBUtilities.timestampMicros()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/service/MigrationTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/MigrationTask.java b/src/java/org/apache/cassandra/service/MigrationTask.java
index 70b917d..ac7c6be 100644
--- a/src/java/org/apache/cassandra/service/MigrationTask.java
+++ b/src/java/org/apache/cassandra/service/MigrationTask.java
@@ -24,7 +24,7 @@ import java.util.Collection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.DefsTable;
 import org.apache.cassandra.db.RowMutation;
 import org.apache.cassandra.gms.FailureDetector;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/service/ReadCallback.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ReadCallback.java b/src/java/org/apache/cassandra/service/ReadCallback.java
index bfd0044..f6cf0bd 100644
--- a/src/java/org/apache/cassandra/service/ReadCallback.java
+++ b/src/java/org/apache/cassandra/service/ReadCallback.java
@@ -21,8 +21,8 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.util.Collections;
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.cassandra.config.Schema;
@@ -37,13 +37,14 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ReadCommand;
 import org.apache.cassandra.db.ReadResponse;
 import org.apache.cassandra.db.Table;
+import org.apache.cassandra.exceptions.ReadTimeoutException;
+import org.apache.cassandra.exceptions.UnavailableException;
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.net.IAsyncCallback;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.thrift.ConsistencyLevel;
-import org.apache.cassandra.thrift.UnavailableException;
+import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.SimpleCondition;
 import org.apache.cassandra.utils.WrappedRunnable;
@@ -63,6 +64,7 @@ public class ReadCallback<TMessage, TResolved> implements IAsyncCallback<TMessag
     protected final int blockfor;
     final List<InetAddress> endpoints;
     private final IReadCommand command;
+    protected final ConsistencyLevel consistencyLevel;
     protected final AtomicInteger received = new AtomicInteger(0);
 
     /**
@@ -71,9 +73,10 @@ public class ReadCallback<TMessage, TResolved> implements IAsyncCallback<TMessag
     public ReadCallback(IResponseResolver<TMessage, TResolved> resolver, ConsistencyLevel consistencyLevel, IReadCommand command, List<InetAddress> endpoints)
     {
         this.command = command;
-        this.blockfor = determineBlockFor(consistencyLevel, command.getKeyspace());
+        this.blockfor = consistencyLevel.blockFor(command.getKeyspace());
         this.resolver = resolver;
         this.startTime = System.currentTimeMillis();
+        this.consistencyLevel = consistencyLevel;
         sortForConsistencyLevel(endpoints);
         this.endpoints = resolver instanceof RowRepairResolver ? endpoints : filterEndpoints(endpoints);
         if (logger.isDebugEnabled())
@@ -127,7 +130,7 @@ public class ReadCallback<TMessage, TResolved> implements IAsyncCallback<TMessag
         return ep.subList(0, Math.min(ep.size(), blockfor));
     }
 
-    public TResolved get() throws TimeoutException, DigestMismatchException, IOException
+    public TResolved get() throws ReadTimeoutException, DigestMismatchException, IOException
     {
         long timeout = command.getTimeout() - (System.currentTimeMillis() - startTime);
         boolean success;
@@ -141,12 +144,7 @@ public class ReadCallback<TMessage, TResolved> implements IAsyncCallback<TMessag
         }
 
         if (!success)
-        {
-            StringBuilder sb = new StringBuilder("");
-            for (MessageIn message : resolver.getMessages())
-                sb.append(message.from).append(", ");
-            throw new TimeoutException("Operation timed out - received only " + received.get() + " responses from " + sb.toString() + " .");
-        }
+            throw new ReadTimeoutException(consistencyLevel, received.get(), blockfor, resolver.isDataPresent());
 
         return blockfor == 1 ? resolver.getData() : resolver.resolve();
     }
@@ -195,32 +193,13 @@ public class ReadCallback<TMessage, TResolved> implements IAsyncCallback<TMessag
         }
     }
 
-    public int determineBlockFor(ConsistencyLevel consistencyLevel, String table)
-    {
-        switch (consistencyLevel)
-        {
-            case ONE:
-                return 1;
-            case TWO:
-                return 2;
-            case THREE:
-                return 3;
-            case QUORUM:
-                return (Table.open(table).getReplicationStrategy().getReplicationFactor() / 2) + 1;
-            case ALL:
-                return Table.open(table).getReplicationStrategy().getReplicationFactor();
-            default:
-                throw new UnsupportedOperationException("invalid consistency level: " + consistencyLevel);
-        }
-    }
-
     public void assureSufficientLiveNodes() throws UnavailableException
     {
         if (endpoints.size() < blockfor)
         {
             logger.debug("Live nodes {} do not satisfy ConsistencyLevel ({} required)",
                          StringUtils.join(endpoints, ", "), blockfor);
-            throw new UnavailableException();
+            throw new UnavailableException(consistencyLevel, blockfor, endpoints.size());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a2faf94/src/java/org/apache/cassandra/service/RepairCallback.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/RepairCallback.java b/src/java/org/apache/cassandra/service/RepairCallback.java
index 8ded3e0..8b010fe 100644
--- a/src/java/org/apache/cassandra/service/RepairCallback.java
+++ b/src/java/org/apache/cassandra/service/RepairCallback.java
@@ -53,7 +53,7 @@ public class RepairCallback implements IAsyncCallback
         this.startTime = System.currentTimeMillis();
     }
 
-    public Row get() throws TimeoutException, DigestMismatchException, IOException
+    public Row get() throws DigestMismatchException, IOException
     {
         long timeout = DatabaseDescriptor.getWriteRpcTimeout() - (System.currentTimeMillis() - startTime);
         try