You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by js...@apache.org on 2015/09/08 19:28:28 UTC

drill git commit: DRILL-3497: Throw UserException#validationError instead of SetOptionException and ExpressionParsingException for options

Repository: drill
Updated Branches:
  refs/heads/master 57c5d15d6 -> 38e660e13


DRILL-3497: Throw UserException#validationError instead of SetOptionException and ExpressionParsingException for options

This closes #98

+ Fixed bug in PositiveLongValidator (0 is not positive)
+ Added UserExceptionMatcher for unit tests

DRILL-3497: Addressing Jason's review comments (1)


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/38e660e1
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/38e660e1
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/38e660e1

Branch: refs/heads/master
Commit: 38e660e13695d341a0cc9e72a058394d0ff9a31d
Parents: 57c5d15
Author: Sudheesh Katkam <sk...@maprtech.com>
Authored: Tue Jul 14 15:33:30 2015 -0700
Committer: Jason Altekruse <al...@gmail.com>
Committed: Tue Sep 8 10:25:30 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/drill/test/DrillTest.java   |  9 ++-
 .../apache/drill/test/UserExceptionMatcher.java | 71 +++++++++++++++++
 .../drill/exec/compile/QueryClassLoader.java    | 12 +--
 .../exec/server/options/BaseOptionManager.java  | 16 ++--
 .../server/options/FallbackOptionManager.java   | 13 +++-
 .../exec/server/options/OptionManager.java      |  2 +-
 .../exec/server/options/OptionValidator.java    |  4 +-
 .../server/options/SessionOptionManager.java    |  2 +-
 .../exec/server/options/SetOptionException.java | 62 ---------------
 .../server/options/SystemOptionManager.java     | 10 ++-
 .../exec/server/options/TypeValidators.java     | 82 +++++++-------------
 .../drill/exec/testing/ExecutionControls.java   | 15 ++--
 .../apache/drill/exec/server/TestOptions.java   | 13 +++-
 13 files changed, 168 insertions(+), 143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/38e660e1/common/src/test/java/org/apache/drill/test/DrillTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/drill/test/DrillTest.java b/common/src/test/java/org/apache/drill/test/DrillTest.java
index 95ba936..18c2c1a 100644
--- a/common/src/test/java/org/apache/drill/test/DrillTest.java
+++ b/common/src/test/java/org/apache/drill/test/DrillTest.java
@@ -23,12 +23,12 @@ import java.lang.management.MemoryMXBean;
 import java.util.List;
 
 import org.apache.drill.common.util.DrillStringUtils;
-import org.apache.drill.common.util.RepeatTestRule;
 import org.apache.drill.common.util.TestTools;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
+import org.junit.rules.ExpectedException;
 import org.junit.rules.TestName;
 import org.junit.rules.TestRule;
 import org.junit.rules.TestWatcher;
@@ -60,6 +60,13 @@ public class DrillTest {
 
   @Rule public final TestRule REPEAT_RULE = TestTools.getRepeatRule(false);
 
+  /**
+   * Rule for tests that verify {@link org.apache.drill.common.exceptions.UserException} type and message. See
+   * {@link UserExceptionMatcher} and e.g. {@link org.apache.drill.exec.server.TestOptions#checkValidationException}.
+   * Tests that do not use this rule are not affected.
+   */
+  @Rule public final ExpectedException thrownException = ExpectedException.none();
+
   @Rule public TestName TEST_NAME = new TestName();
 
   @Before

http://git-wip-us.apache.org/repos/asf/drill/blob/38e660e1/common/src/test/java/org/apache/drill/test/UserExceptionMatcher.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/drill/test/UserExceptionMatcher.java b/common/src/test/java/org/apache/drill/test/UserExceptionMatcher.java
new file mode 100644
index 0000000..eba0d78
--- /dev/null
+++ b/common/src/test/java/org/apache/drill/test/UserExceptionMatcher.java
@@ -0,0 +1,71 @@
+/**
+ * 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.drill.test;
+
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType;
+import org.hamcrest.Description;
+import org.hamcrest.TypeSafeMatcher;
+
+/**
+ * Matcher for UserException that matches if expected type and actual type are the same, and expected message is
+ * contained in the actual message.
+ */
+public class UserExceptionMatcher extends TypeSafeMatcher<UserException> {
+
+  private final ErrorType expectedType;
+  private final String expectedMessage;
+
+  public UserExceptionMatcher(final ErrorType expectedType, final String expectedMessage) {
+    this.expectedType = expectedType;
+    this.expectedMessage = expectedMessage;
+  }
+
+  public UserExceptionMatcher(final ErrorType expectedType) {
+    this(expectedType, null);
+  }
+
+  @Override
+  protected boolean matchesSafely(final UserException e) {
+    // Use .contains(...) to compare expected and actual message as the exact messages may differ.
+    return expectedType == e.getErrorType() && (expectedMessage == null || e.getMessage().contains(expectedMessage));
+  }
+
+  @Override
+  public void describeTo(final Description description) {
+    description.appendText("UserException of type: ")
+      .appendValue(expectedType.toString());
+    if (expectedMessage != null) {
+      description.appendText(" with message that contains: \"")
+        .appendText(expectedMessage)
+        .appendText("\"");
+    }
+  }
+
+  @Override
+  protected void describeMismatchSafely(final UserException e, final Description description) {
+    description.appendText("UserException thrown was of type: ")
+      .appendValue(e.getErrorType().toString());
+    if (expectedMessage != null) {
+      description.appendText(" with message: \"")
+        .appendText(e.getMessage())
+      .appendText("\"");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/38e660e1/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
index d49b500..c4eaae8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
@@ -25,7 +25,7 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.exceptions.ExpressionParsingException;
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.compile.ClassTransformer.ClassNames;
 import org.apache.drill.exec.exception.ClassTransformationException;
 import org.apache.drill.exec.server.options.OptionManager;
@@ -39,18 +39,20 @@ import org.codehaus.commons.compiler.CompileException;
 import com.google.common.collect.MapMaker;
 
 public class QueryClassLoader extends URLClassLoader {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryClassLoader.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryClassLoader.class);
 
   public static final String JAVA_COMPILER_OPTION = "exec.java_compiler";
   public static final StringValidator JAVA_COMPILER_VALIDATOR = new StringValidator(JAVA_COMPILER_OPTION, CompilerPolicy.DEFAULT.toString()) {
     @Override
-    public void validate(OptionValue v) throws ExpressionParsingException {
+    public void validate(OptionValue v) {
       super.validate(v);
       try {
         CompilerPolicy.valueOf(v.string_val.toUpperCase());
       } catch (IllegalArgumentException e) {
-        throw new ExpressionParsingException(String.format("Invalid value '%s' specified for option '%s'. Valid values are %s.",
-            v.string_val, getOptionName(), Arrays.toString(CompilerPolicy.values())));
+        throw UserException.validationError()
+            .message("Invalid value '%s' specified for option '%s'. Valid values are %s.",
+              v.string_val, getOptionName(), Arrays.toString(CompilerPolicy.values()))
+            .build(logger);
       }
     }
   };

http://git-wip-us.apache.org/repos/asf/drill/blob/38e660e1/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java
index e973318..bbcdec8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java
@@ -25,13 +25,15 @@ import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
 abstract class BaseOptionManager implements OptionManager {
 //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseOptionManager.class);
 
-
-  private OptionValue getOptionSafe(OptionValidator validator){
-    OptionValue value = getOption(validator.getOptionName());
-    if(value == null){
-      throw new IllegalArgumentException(String.format("Unknown value for option `%s`.", validator.getOptionName()));
-    }
-    return value;
+  /**
+   * Gets the current option value given a validator.
+   *
+   * @param validator the validator
+   * @return option value
+   * @throws IllegalArgumentException - if the validator is not found
+   */
+  private OptionValue getOptionSafe(OptionValidator validator)  {
+    return getOption(validator.getOptionName());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/38e660e1/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java
index d96045c..7c864b2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.server.options;
 import java.util.Iterator;
 
 import com.google.common.collect.Iterables;
+import org.apache.drill.common.exceptions.UserException;
 
 /**
  * An {@link OptionManager} which allows for falling back onto another {@link OptionManager}. This way method calls can
@@ -31,7 +32,7 @@ import com.google.common.collect.Iterables;
  * manager. {@link QueryOptionManager} uses {@link SessionOptionManager} as the fall back manager.
  */
 public abstract class FallbackOptionManager extends BaseOptionManager {
-//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FallbackOptionManager.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FallbackOptionManager.class);
 
   protected final OptionManager fallback;
 
@@ -84,7 +85,15 @@ public abstract class FallbackOptionManager extends BaseOptionManager {
 
   @Override
   public void setOption(OptionValue value) {
-    SystemOptionManager.getValidator(value.name).validate(value); // validate the option
+    final OptionValidator validator;
+    try {
+      validator = SystemOptionManager.getValidator(value.name);
+    } catch (final IllegalArgumentException e) {
+      throw UserException.validationError()
+        .message(e.getMessage())
+        .build(logger);
+    }
+    validator.validate(value); // validate the option
 
     // fallback if unable to set locally
     if (!setLocalOption(value)) {

http://git-wip-us.apache.org/repos/asf/drill/blob/38e660e1/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java
index 45535ae..8ff0f94 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java
@@ -26,7 +26,7 @@ public interface OptionManager extends Iterable<OptionValue> {
    * Sets an option value.
    *
    * @param value option value
-   * @throws IllegalArgumentException message to describe error with value
+   * @throws org.apache.drill.common.exceptions.UserException message to describe error with value
    */
   void setOption(OptionValue value);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/38e660e1/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValidator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValidator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValidator.java
index 675c354..3b43f9a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValidator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValidator.java
@@ -17,7 +17,7 @@
  ******************************************************************************/
 package org.apache.drill.exec.server.options;
 
-import org.apache.drill.common.exceptions.ExpressionParsingException;
+import org.apache.drill.common.exceptions.UserException;
 
 /**
  * Validates the values provided to Drill options.
@@ -79,7 +79,7 @@ public abstract class OptionValidator {
    * Validates the option value.
    *
    * @param value the value to validate
-   * @throws ExpressionParsingException message to describe error with value
+   * @throws UserException message to describe error with value, including range or list of expected values
    */
   public abstract void validate(OptionValue value);
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/38e660e1/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SessionOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SessionOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SessionOptionManager.java
index d2bc82f..eb0da03 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SessionOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SessionOptionManager.java
@@ -52,7 +52,7 @@ public class SessionOptionManager extends InMemoryOptionManager {
       return false;
     }
     final String name = value.name;
-    final OptionValidator validator = SystemOptionManager.getValidator(name);
+    final OptionValidator validator = SystemOptionManager.getValidator(name); // if set, validator must exist.
     final boolean shortLived = validator.isShortLived();
     if (shortLived) {
       final int start = session.getQueryCount() + 1; // start from the next query

http://git-wip-us.apache.org/repos/asf/drill/blob/38e660e1/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SetOptionException.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SetOptionException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SetOptionException.java
deleted file mode 100644
index dd698c3..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SetOptionException.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * 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.drill.exec.server.options;
-
-import java.util.Set;
-
-import javax.validation.ConstraintViolation;
-
-import org.apache.drill.common.exceptions.LogicalPlanParsingException;
-import org.apache.drill.common.logical.data.LogicalOperator;
-import org.apache.drill.common.logical.data.LogicalOperatorBase;
-
-public class SetOptionException extends LogicalPlanParsingException{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SetOptionException.class);
-
-  public SetOptionException() {
-    super();
-
-  }
-
-  public SetOptionException(LogicalOperator operator, Set<ConstraintViolation<LogicalOperatorBase>> violations) {
-    super(operator, violations);
-
-  }
-
-  public SetOptionException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
-    super(message, cause, enableSuppression, writableStackTrace);
-
-  }
-
-  public SetOptionException(String message, Throwable cause) {
-    super(message, cause);
-
-  }
-
-  public SetOptionException(String message) {
-    super(message);
-
-  }
-
-  public SetOptionException(Throwable cause) {
-    super(cause);
-
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/38e660e1/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index 2df36dd..1b9906d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -26,6 +26,7 @@ import java.util.Map.Entry;
 import org.apache.commons.collections.IteratorUtils;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.compile.ClassTransformer;
 import org.apache.drill.exec.compile.QueryClassLoader;
@@ -221,7 +222,14 @@ public class SystemOptionManager extends BaseOptionManager {
   public void setOption(final OptionValue value) {
     assert value.type == OptionType.SYSTEM;
     final String name = value.name.toLowerCase();
-    final OptionValidator validator = getValidator(name);
+    final OptionValidator validator;
+    try {
+      validator = getValidator(name);
+    } catch (final IllegalArgumentException e) {
+      throw UserException.validationError()
+        .message(e.getMessage())
+        .build(logger);
+    }
     validator.validate(value); // validate the option
 
     if (options.get(name) == null && value.equals(validator.getDefault())) {

http://git-wip-us.apache.org/repos/asf/drill/blob/38e660e1/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java
index 829958a..73f067b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java
@@ -17,17 +17,15 @@
  */
 package org.apache.drill.exec.server.options;
 
-import java.io.IOException;
 import java.util.HashSet;
 import java.util.Set;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.drill.common.exceptions.ExpressionParsingException;
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.server.options.OptionValue.Kind;
 import org.apache.drill.exec.server.options.OptionValue.OptionType;
 
 public class TypeValidators {
-//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeValidators.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeValidators.class);
 
   public static class PositiveLongValidator extends LongValidator {
     private final long max;
@@ -38,11 +36,12 @@ public class TypeValidators {
     }
 
     @Override
-    public void validate(OptionValue v) throws ExpressionParsingException {
+    public void validate(OptionValue v) {
       super.validate(v);
-      if (v.num_val > max || v.num_val < 0) {
-        throw new ExpressionParsingException(String.format("Option %s must be between %d and %d.", getOptionName(), 0,
-            max));
+      if (v.num_val > max || v.num_val < 1) {
+        throw UserException.validationError()
+            .message(String.format("Option %s must be between %d and %d.", getOptionName(), 1, max))
+            .build(logger);
       }
     }
   }
@@ -54,14 +53,16 @@ public class TypeValidators {
     }
 
     @Override
-    public void validate(OptionValue v) throws ExpressionParsingException {
+    public void validate(OptionValue v) {
       super.validate(v);
       if (!isPowerOfTwo(v.num_val)) {
-        throw new ExpressionParsingException(String.format("Option %s must be a power of two.", getOptionName()));
+        throw UserException.validationError()
+            .message(String.format("Option %s must be a power of two.", getOptionName()))
+            .build(logger);
       }
     }
 
-    private boolean isPowerOfTwo(long num) {
+    private static boolean isPowerOfTwo(long num) {
       return (num & (num - 1)) == 0;
     }
   }
@@ -77,14 +78,14 @@ public class TypeValidators {
     }
 
     @Override
-    public void validate(OptionValue v) throws ExpressionParsingException {
+    public void validate(OptionValue v) {
       super.validate(v);
       if (v.float_val > max || v.float_val < min) {
-        throw new ExpressionParsingException(String.format("Option %s must be between %f and %f.",
-            getOptionName(), min, max));
+        throw UserException.validationError()
+            .message(String.format("Option %s must be between %f and %f.", getOptionName(), min, max))
+            .build(logger);
       }
     }
-
   }
 
   public static class BooleanValidator extends TypeValidator {
@@ -122,11 +123,12 @@ public class TypeValidators {
     }
 
     @Override
-    public void validate(OptionValue v) throws ExpressionParsingException {
+    public void validate(OptionValue v) {
       super.validate(v);
       if (v.num_val > max || v.num_val < min) {
-        throw new ExpressionParsingException(String.format("Option %s must be between %d and %d.",
-            getOptionName(), min, max));
+        throw UserException.validationError()
+            .message(String.format("Option %s must be between %d and %d.", getOptionName(), min, max))
+            .build(logger);
       }
     }
   }
@@ -145,39 +147,12 @@ public class TypeValidators {
     }
 
     @Override
-    public void validate(final OptionValue v) throws ExpressionParsingException {
+    public void validate(final OptionValue v) {
       super.validate(v);
       if (!valuesSet.contains(v.string_val.toLowerCase())) {
-        throw new ExpressionParsingException(String.format("Option %s must be one of: %s", getOptionName(), valuesSet));
-      }
-    }
-  }
-
-  /**
-   * Validator for POJO passed in as JSON string
-   */
-  public static class JsonStringValidator extends StringValidator {
-
-    private static final ObjectMapper mapper = new ObjectMapper();
-    private final Class<?> clazz;
-
-    public JsonStringValidator(final String name, final Class<?> clazz, final String def) {
-      super(name, def);
-      this.clazz = clazz;
-      validateJson(def, clazz);
-    }
-
-    @Override
-    public void validate(final OptionValue v) throws ExpressionParsingException {
-      super.validate(v);
-      validateJson(v.string_val, clazz);
-    }
-
-    private static void validateJson(final String jsonString, final Class<?> clazz) {
-      try {
-        mapper.readValue(jsonString, clazz);
-      } catch (IOException e) {
-        throw new ExpressionParsingException("Invalid JSON string (" + jsonString + ") for class " + clazz.getName(), e);
+        throw UserException.validationError()
+            .message(String.format("Option %s must be one of: %s.", getOptionName(), valuesSet))
+            .build(logger);
       }
     }
   }
@@ -198,11 +173,12 @@ public class TypeValidators {
     }
 
     @Override
-    public void validate(final OptionValue v) throws ExpressionParsingException {
+    public void validate(final OptionValue v) {
       if (v.kind != kind) {
-        throw new ExpressionParsingException(String.format(
-            "Option %s must be of type %s but you tried to set to %s.",
-            getOptionName(), kind.name(), v.kind.name()));
+        throw UserException.validationError()
+            .message(String.format("Option %s must be of type %s but you tried to set to %s.", getOptionName(),
+              kind.name(), v.kind.name()))
+            .build(logger);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/38e660e1/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java
index 2c0afe4..8f9589d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java
@@ -20,11 +20,10 @@ package org.apache.drill.exec.testing;
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonSubTypes.Type;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import com.fasterxml.jackson.core.JsonParseException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
-import org.apache.drill.common.exceptions.ExpressionParsingException;
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.server.options.OptionManager;
@@ -81,7 +80,7 @@ public final class ExecutionControls {
      * @param ttl  the number of queries for which this option should be valid
      */
     public ControlsOptionValidator(final String name, final String def, final int ttl) {
-      super(name, OptionValue.Kind.DOUBLE, OptionValue.createString(OptionType.SESSION, name, def));
+      super(name, OptionValue.Kind.STRING, OptionValue.createString(OptionType.SESSION, name, def));
       assert ttl > 0;
       this.ttl = ttl;
     }
@@ -97,15 +96,19 @@ public final class ExecutionControls {
     }
 
     @Override
-    public void validate(final OptionValue v) throws ExpressionParsingException {
+    public void validate(final OptionValue v) {
       if (v.type != OptionType.SESSION) {
-        throw new ExpressionParsingException("Controls can be set only at SESSION level.");
+        throw UserException.validationError()
+            .message("Controls can be set only at SESSION level.")
+            .build(logger);
       }
       final String jsonString = v.string_val;
       try {
         validateControlsString(jsonString);
       } catch (final IOException e) {
-        throw new ExpressionParsingException("Invalid control options string (" + jsonString + ").", e);
+        throw UserException.validationError()
+            .message(String.format("Invalid controls option string (%s) due to %s.", jsonString, e.getMessage()))
+            .build(logger);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/38e660e1/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptions.java
index 71ac653..f20fd25 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptions.java
@@ -19,10 +19,13 @@ package org.apache.drill.exec.server;
 
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.test.UserExceptionMatcher;
 import org.junit.Test;
 
+import static org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType.VALIDATION;
+
 public class TestOptions extends BaseTestQuery{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestOptions.class);
+//  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestOptions.class);
 
   @Test
   public void testDrillbits() throws Exception{
@@ -37,7 +40,13 @@ public class TestOptions extends BaseTestQuery{
         "select * from sys.options;" +
         "ALTER SESSION set `planner.disable_exchanges` = true;" +
         "select * from sys.options;"
-        );
+    );
+  }
+
+  @Test
+  public void checkValidationException() throws Exception {
+    thrownException.expect(new UserExceptionMatcher(VALIDATION));
+    test(String.format("ALTER session SET `%s` = '%s';", ExecConstants.SLICE_TARGET, "fail"));
   }
 
   @Test // DRILL-3122