You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by bu...@apache.org on 2017/01/05 01:12:25 UTC

asterixdb git commit: Use a file to store the map from error codes to error messages.

Repository: asterixdb
Updated Branches:
  refs/heads/master 8b2aceeb9 -> 8a29c5f64


Use a file to store the map from error codes to error messages.

Change-Id: I344903a9961469b62a951a9095514bf671ff11ee
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1410
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
BAD: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>


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

Branch: refs/heads/master
Commit: 8a29c5f649ffc19126a38b9c8cd48a2fded3d01b
Parents: 8b2acee
Author: Yingyi Bu <yi...@couchbase.com>
Authored: Wed Jan 4 11:56:06 2017 -0800
Committer: Yingyi Bu <bu...@gmail.com>
Committed: Wed Jan 4 17:11:59 2017 -0800

----------------------------------------------------------------------
 asterixdb/asterix-common/pom.xml                |  1 +
 .../LSMInsertDeleteOperatorNodePushable.java    | 10 +--
 .../asterix/common/exceptions/ErrorCode.java    | 79 +++++---------------
 .../main/resources/asx_errormsg/en.properties   | 50 +++++++++++++
 .../exceptions/InvalidExpressionException.java  |  4 +-
 .../runtime/exceptions/UnderflowException.java  |  4 +-
 .../operators/std/AssignRuntimeFactory.java     |  5 +-
 .../hyracks/api/exceptions/ErrorCode.java       | 31 +++++++-
 .../api/exceptions/HyracksDataException.java    |  4 +
 .../api/exceptions/ProcessTupleException.java   | 34 +++++++++
 .../hyracks/api/util/ErrorMessageUtil.java      | 41 ++++++++++
 .../src/main/resources/errormsg/en.properties   | 21 ++++++
 12 files changed, 207 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8a29c5f6/asterixdb/asterix-common/pom.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/pom.xml b/asterixdb/asterix-common/pom.xml
index 82cb575..d4a4517 100644
--- a/asterixdb/asterix-common/pom.xml
+++ b/asterixdb/asterix-common/pom.xml
@@ -156,6 +156,7 @@
         <configuration>
           <includes>
             <include>**/*.class</include>
+            <include>**/*.properties</include>
             <include>**/README*</include>
             <include>**/NOTICE*</include>
             <include>**/LICENSE*</include>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8a29c5f6/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java
index f17fcaf..2cae9de 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java
@@ -21,14 +21,15 @@ package org.apache.asterix.common.dataflow;
 import java.nio.ByteBuffer;
 
 import org.apache.asterix.common.api.IAppRuntimeContext;
+import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.asterix.common.transactions.ILogMarkerCallback;
 import org.apache.asterix.common.transactions.PrimaryIndexLogMarkerCallback;
 import org.apache.hyracks.api.comm.VSizeFrame;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.ProcessTupleException;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
@@ -135,8 +136,7 @@ public class LSMInsertDeleteOperatorNodePushable extends LSMIndexInsertUpdateDel
                         }
                         break;
                     default: {
-                        throw new HyracksDataException("Unsupported operation %1$s in %2$s operator",
-                                ErrorCode.INVALID_OPERATOR_OPERATION, op.toString(),
+                        throw HyracksDataException.create(ErrorCode.INVALID_OPERATOR_OPERATION, op.toString(),
                                 LSMInsertDeleteOperatorNodePushable.class.getSimpleName());
                     }
                 }
@@ -145,10 +145,10 @@ public class LSMInsertDeleteOperatorNodePushable extends LSMIndexInsertUpdateDel
             if (e.getErrorCode() == ErrorCode.INVALID_OPERATOR_OPERATION) {
                 throw e;
             } else {
-                throw new HyracksDataException(e, ErrorCode.ERROR_PROCESSING_TUPLE, i);
+                throw new ProcessTupleException(e, i);
             }
         } catch (Exception e) {
-            throw new HyracksDataException(e, ErrorCode.ERROR_PROCESSING_TUPLE, i);
+            throw new ProcessTupleException(e, i);
         }
 
         writeBuffer.ensureFrameSize(buffer.capacity());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8a29c5f6/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index d5dfd87..2e7c69b 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -18,9 +18,12 @@
  */
 package org.apache.asterix.common.exceptions;
 
-import java.util.HashMap;
+import java.io.File;
+import java.io.InputStream;
 import java.util.Map;
 
+import org.apache.hyracks.api.util.ErrorMessageUtil;
+
 // Error code:
 // 0 --- 999:  runtime errors
 // 1000 ---- 1999: compilation errors
@@ -28,6 +31,7 @@ import java.util.Map;
 // 3000 ---- 3999: feed errors
 // 4000 ---- 4999: lifecycle management errors
 public class ErrorCode {
+    private static final String RESOURCE_PATH = "asx_errormsg" + File.separator + "en.properties";
     public static final String ASTERIX = "ASX";
 
     // Extension errors
@@ -55,74 +59,25 @@ public class ErrorCode {
     public static final int ERROR_COMPILATION_TYPE_INCOMPATIBLE = 1003;
     public static final int ERROR_COMPILATION_TYPE_UNSUPPORTED = 1004;
     public static final int ERROR_COMPILATION_TYPE_ITEM = 1005;
-    public static final int ERROR_COMPILATION_INVALID_EXPRESSION = 1006;
-    public static final int ERROR_COMPILATION_INVALID_PARAMETER_NUMBER = 1007;
-    public static final int ERROR_COMPILATION_DUPLICATE_FIELD_NAME = 1008;
+    public static final int ERROR_COMPILATION_DUPLICATE_FIELD_NAME = 1006;
+    public static final int ERROR_COMPILATION_INVALID_EXPRESSION = 1007;
+    public static final int ERROR_COMPILATION_INVALID_PARAMETER_NUMBER = 1008;
     public static final int ERROR_COMPILATION_INVALID_RETURNING_EXPRESSION = 1009;
 
-    private static final String ERROR_MESSAGE_ID_CONFLICT = "Two Extensions share the same Id: %1$s";
-    private static final String ERROR_MESSAGE_COMPONENT_CONFLICT = "Extension Conflict between %1$s and %2$s both "
-            + "extensions extend %3$s";
-    private static final String ERROR_MESSAGE_TYPE_MISMATCH = "Type mismatch: function %1$s expects"
-            + " its %2$s input parameter to be type %3$s, but the actual input type is %4$s";
-    private static final String ERROR_MESSAGE_TYPE_INCOMPATIBLE = "Type incompatibility: function %1$s gets"
-            + " incompatible input values: %2$s and %3$s";
-    private static final String ERROR_MESSAGE_TYPE_UNSUPPORTED = "Unsupported type: %1$s"
-            + " cannot process input type %2$s";
-    private static final String ERROR_MESSAGE_TYPE_ITEM = "Invalid item type: function %1$s"
-            + " cannot process item type %2$s in an input array (or multiset)";
-    private static final String ERROR_MESSAGE_INVALID_FORMAT = "Invalid format for %1$s in %2$s";
-    private static final String ERROR_MESSAGE_OVERFLOW = "Overflow happend in %1$s";
-    private static final String ERROR_MESSAGE_UNDERFLOW = "Underflow happend in %1$s";
-    private static final String ERROR_MESSAGE_INJECTED_FAILURE = "Injected failure in %1$s";
-    private static final String ERROR_MESSAGE_NEGATIVE_VALUE = "Invalid value: function %1$s expects"
-            + " its %2$s input parameter to be a non-negative value, but gets %3$s";
-    private static final String ERROR_MESSAGE_OUT_OF_BOUND = "Index out of bound in %1$s: %2$s";
-    private static final String ERROR_MESSAGE_COERCION = "Invalid implicit scalar to collection coercion in %1$s";
-    private static final String ERROR_MESSAGE_DUPLICATE_FIELD = "Duplicate field name \"%1$s\"";
-    private static final String ERROR_MESSAGE_INVALID_EXPRESSION = "Invalid expression: function %1$s expects"
-            + " its %2$s input parameter to be a %3$s expression, but the actual expression is %4$s";
-    private static final String ERROR_MESSAGE_INVALID_PARAMETER_NUMBER = "Invalid parameter number: function %1$s "
-            + "cannot take %2$s parameters";
-    private static final String ERROR_MESSAGE_INVALID_RETURNING_EXPRESSION = "A returning expression cannot"
-            + " contain dataset access";
-
-    private static Map<Integer, String> errorMessageMap = new HashMap<>();
-
-    static {
-        // runtime errors
-        errorMessageMap.put(ERROR_TYPE_MISMATCH, ERROR_MESSAGE_TYPE_MISMATCH);
-        errorMessageMap.put(ERROR_TYPE_INCOMPATIBLE, ERROR_MESSAGE_TYPE_INCOMPATIBLE);
-        errorMessageMap.put(ERROR_TYPE_ITEM, ERROR_MESSAGE_TYPE_ITEM);
-        errorMessageMap.put(ERROR_TYPE_UNSUPPORTED, ERROR_MESSAGE_TYPE_UNSUPPORTED);
-        errorMessageMap.put(ERROR_INVALID_FORMAT, ERROR_MESSAGE_INVALID_FORMAT);
-        errorMessageMap.put(ERROR_OVERFLOW, ERROR_MESSAGE_OVERFLOW);
-        errorMessageMap.put(ERROR_UNDERFLOW, ERROR_MESSAGE_UNDERFLOW);
-        errorMessageMap.put(ERROR_INJECTED_FAILURE, ERROR_MESSAGE_INJECTED_FAILURE);
-        errorMessageMap.put(ERROR_NEGATIVE_VALUE, ERROR_MESSAGE_NEGATIVE_VALUE);
-        errorMessageMap.put(ERROR_OUT_OF_BOUND, ERROR_MESSAGE_OUT_OF_BOUND);
-        errorMessageMap.put(ERROR_COERCION, ERROR_MESSAGE_COERCION);
-        errorMessageMap.put(ERROR_DUPLICATE_FIELD_NAME, ERROR_MESSAGE_DUPLICATE_FIELD);
-
-        // compilation errors
-        errorMessageMap.put(ERROR_COMPILATION_TYPE_MISMATCH, ERROR_MESSAGE_TYPE_MISMATCH);
-        errorMessageMap.put(ERROR_COMPILATION_TYPE_INCOMPATIBLE, ERROR_MESSAGE_TYPE_INCOMPATIBLE);
-        errorMessageMap.put(ERROR_COMPILATION_TYPE_ITEM, ERROR_MESSAGE_TYPE_ITEM);
-        errorMessageMap.put(ERROR_COMPILATION_TYPE_UNSUPPORTED, ERROR_MESSAGE_TYPE_UNSUPPORTED);
-        errorMessageMap.put(ERROR_COMPILATION_INVALID_EXPRESSION, ERROR_MESSAGE_INVALID_EXPRESSION);
-        errorMessageMap.put(ERROR_COMPILATION_INVALID_PARAMETER_NUMBER, ERROR_MESSAGE_INVALID_PARAMETER_NUMBER);
-        errorMessageMap.put(ERROR_COMPILATION_DUPLICATE_FIELD_NAME, ERROR_MESSAGE_DUPLICATE_FIELD);
-        errorMessageMap.put(ERROR_COMPILATION_INVALID_RETURNING_EXPRESSION, ERROR_MESSAGE_INVALID_RETURNING_EXPRESSION);
-
-        // lifecycle management errors
-        errorMessageMap.put(ERROR_EXTENSION_ID_CONFLICT, ERROR_MESSAGE_ID_CONFLICT);
-        errorMessageMap.put(ERROR_EXTENSION_COMPONENT_CONFLICT, ERROR_MESSAGE_COMPONENT_CONFLICT);
-    }
+    // Loads the map that maps error codes to error message templates.
+    private static Map<Integer, String> errorMessageMap = null;
 
     private ErrorCode() {
     }
 
     public static String getErrorMessage(int errorCode) {
+        if (errorMessageMap == null) {
+            try (InputStream resourceStream = ErrorCode.class.getClassLoader().getResourceAsStream(RESOURCE_PATH)) {
+                errorMessageMap = ErrorMessageUtil.loadErrorMap(resourceStream);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
         String msg = errorMessageMap.get(errorCode);
         if (msg == null) {
             throw new IllegalStateException("Undefined error code: " + errorCode);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8a29c5f6/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
new file mode 100644
index 0000000..85fa1d1
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -0,0 +1,50 @@
+#
+# 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.
+#
+
+# Error code:
+# 0 --- 999:  runtime errors
+# 1000 ---- 1999: compilation errors
+# 2000 ---- 2999: storage errors
+# 3000 ---- 3999: feed errors
+# 4000 ---- 4999: lifecycle management errors
+
+# For the extension lifecycle
+4001 = Two Extensions share the same Id: %1$s
+4002 = Extension Conflict between %1$s and %2$s both extensions extend %3$s
+
+# Type errors
+2,1002 = Type mismatch: function %1$s expects its %2$s input parameter to be type %3$s, but the actual input type is %4$s
+3,1003 = Type incompatibility: function %1$s gets incompatible input values: %2$s and %3$s
+4,1004 = Unsupported type: %1$s cannot process input type %2$s
+5,1005 = Invalid item type: function %1$s cannot process item type %2$s in an input array (or multiset)
+13,1006=Duplicate field name \"%1$s\"
+1009=A returning expression cannot contain dataset access
+
+# Data errors
+6 = Invalid format for %1$s in %2$s
+7 = Overflow happend in %1$s
+8=Underflow happend in %1$s
+9=Injected failure in %1$s
+10=Invalid value: function %1$s expects its %2$s input parameter to be a non-negative value, but gets %3$s
+11=Index out of bound in %1$s: %2$s
+12=Invalid implicit scalar to collection coercion in %1$s
+
+# Compile-time check errors
+1007=Invalid expression: function %1$s expects its %2$s input parameter to be a %3$s expression, but the actual expression is %4$s
+1008=Invalid parameter number: function %1$s cannot take %2$s parameters

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8a29c5f6/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/InvalidExpressionException.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/InvalidExpressionException.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/InvalidExpressionException.java
index a4b3a32..c381b7b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/InvalidExpressionException.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/InvalidExpressionException.java
@@ -34,14 +34,14 @@ public class InvalidExpressionException extends CompilationException {
 
     public InvalidExpressionException(FunctionIdentifier fid, int index, ILogicalExpression actualExpr,
             LogicalExpressionTag... exprKinds) {
-        super(ErrorCode.ERROR_COMPILATION_TYPE_INCOMPATIBLE, fid.getName(), indexToPosition(index),
+        super(ErrorCode.ERROR_COMPILATION_INVALID_EXPRESSION, fid.getName(), indexToPosition(index),
                 actualExpr.toString(),
                 toExpectedTypeString(exprKinds));
     }
 
     public InvalidExpressionException(String functionName, int index, ILogicalExpression actualExpr,
             LogicalExpressionTag... exprKinds) {
-        super(ErrorCode.ERROR_COMPILATION_TYPE_INCOMPATIBLE, functionName, indexToPosition(index),
+        super(ErrorCode.ERROR_COMPILATION_INVALID_EXPRESSION, functionName, indexToPosition(index),
                 actualExpr.toString(),
                 toExpectedTypeString(exprKinds));
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8a29c5f6/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnderflowException.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnderflowException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnderflowException.java
index fede04f..cb39ab2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnderflowException.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnderflowException.java
@@ -27,12 +27,12 @@ public class UnderflowException extends RuntimeDataException {
 
     // Underflow.
     public UnderflowException(FunctionIdentifier fid) {
-        super(ErrorCode.ERROR_OVERFLOW, fid.getName());
+        super(ErrorCode.ERROR_UNDERFLOW, fid.getName());
     }
 
     // Underflow.
     public UnderflowException(String functionName) {
-        super(ErrorCode.ERROR_OVERFLOW, functionName);
+        super(ErrorCode.ERROR_UNDERFLOW, functionName);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8a29c5f6/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
index 520465a..0e33a7a 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
@@ -30,6 +30,7 @@ import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.ProcessTupleException;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -173,9 +174,7 @@ public class AssignRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactor
                         }
                     }
                 } catch (HyracksDataException e) {
-                    throw new HyracksDataException(ErrorCode.HYRACKS, ErrorCode.ERROR_PROCESSING_TUPLE,
-                            "Error evaluating tuple %1$s in AssignRuntime", (Throwable) e,
-                            new Serializable[] { tupleIndex });
+                    throw new ProcessTupleException(e, tupleIndex);
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8a29c5f6/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
index c95c31c..95f2e8e 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
@@ -18,16 +18,41 @@
  */
 package org.apache.hyracks.api.exceptions;
 
+import java.io.File;
+import java.io.InputStream;
+import java.util.Map;
+
+import org.apache.hyracks.api.util.ErrorMessageUtil;
+
 /**
  * A registry of runtime error codes
  */
 public class ErrorCode {
+    private static final String RESOURCE_PATH = "errormsg" + File.separator + "en.properties";
     public static final String HYRACKS = "HYR";
-    public static final int ERROR_PROCESSING_TUPLE = 0;
+
     public static final int INVALID_OPERATOR_OPERATION = 1;
-    public static final int FAILURE_ON_NODE = 2;
-    public static final int ILLEGAL_ARGUMENT = 3;
+    public static final int ERROR_PROCESSING_TUPLE = 2;
+    public static final int FAILURE_ON_NODE = 3;
+
+    // Loads the map that maps error codes to error message templates.
+    private static Map<Integer, String> errorMessageMap = null;
 
     private ErrorCode() {
     }
+
+    public static String getErrorMessage(int errorCode) {
+        if (errorMessageMap == null) {
+            try (InputStream resourceStream = ErrorCode.class.getClassLoader().getResourceAsStream(RESOURCE_PATH)) {
+                errorMessageMap = ErrorMessageUtil.loadErrorMap(resourceStream);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+        String msg = errorMessageMap.get(errorCode);
+        if (msg == null) {
+            throw new IllegalStateException("Undefined error code: " + errorCode);
+        }
+        return msg;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8a29c5f6/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
index 8a0e94b..d3e4f2e 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
@@ -36,6 +36,10 @@ public class HyracksDataException extends HyracksException {
     private final String nodeId;
     private transient volatile String msgCache;
 
+    public static HyracksDataException create(int code, Serializable... params) {
+        return new HyracksDataException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), params);
+    }
+
     public HyracksDataException(String component, int errorCode, String message, Throwable cause, String nodeId,
             Serializable... params) {
         super(message, cause);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8a29c5f6/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ProcessTupleException.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ProcessTupleException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ProcessTupleException.java
new file mode 100644
index 0000000..67c03e3
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ProcessTupleException.java
@@ -0,0 +1,34 @@
+/*
+ * 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.hyracks.api.exceptions;
+
+import java.io.Serializable;
+
+public class ProcessTupleException extends HyracksDataException {
+
+    public ProcessTupleException(Serializable... params) {
+        super(ErrorCode.HYRACKS, ErrorCode.ERROR_PROCESSING_TUPLE,
+                ErrorCode.getErrorMessage(ErrorCode.ERROR_PROCESSING_TUPLE), params);
+    }
+
+    public ProcessTupleException(Throwable cause, Serializable... params) {
+        super(ErrorCode.HYRACKS, ErrorCode.ERROR_PROCESSING_TUPLE, cause, params);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8a29c5f6/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
index fd1e850..c2f5141 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
@@ -21,21 +21,62 @@
 
 package org.apache.hyracks.api.util;
 
+import java.io.IOError;
+import java.io.IOException;
+import java.io.InputStream;
 import java.io.Serializable;
 import java.util.Formatter;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import org.apache.hyracks.api.exceptions.ErrorCode;
+
 public class ErrorMessageUtil {
 
     private static final Logger LOGGER = Logger.getLogger(ErrorMessageUtil.class.getName());
     public static final String NONE = "";
+    private static final String COMMA = ",";
 
     private ErrorMessageUtil() {
 
     }
 
     /**
+     * Loads the mapping from error codes to error messages from a properties file.
+     * The key of properties is in the form of comma-separated error codes and the value is the corresponding
+     * error message template.
+     *
+     * Example entries in the properties file:
+     * 4002=Extension Conflict between %1$s and %2$s both extensions extend %3$s
+     * 2,1002 = Type mismatch: function %1$s expects its %2$s input parameter to be type %3$s
+     *
+     * @param resourceStream,
+     *            the input stream of the properties file
+     * @return the map that maps error codes (integers) to error message templates (strings).
+     */
+    public static Map<Integer, String> loadErrorMap(InputStream resourceStream) throws IOException {
+        Properties prop = new Properties();
+        Map<Integer, String> errorMessageMap = new HashMap<>();
+        prop.load(resourceStream);
+        for (Map.Entry<Object, Object> entry : prop.entrySet()) {
+            String key = (String) entry.getKey();
+            String msg = (String) entry.getValue();
+            if (key.contains(COMMA)) {
+                String[] codes = key.split(COMMA);
+                for (String code : codes) {
+                    errorMessageMap.put(Integer.parseInt(code), msg);
+                }
+            } else {
+                errorMessageMap.put(Integer.parseInt(key), msg);
+            }
+        }
+        return errorMessageMap;
+    }
+
+    /**
      * formats a error message
      * Example:
      * formatMessage(HYRACKS, ErrorCode.UNKNOWN, "%1$s -- %2$s", "one", "two") returns "HYR0000: one -- two"

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8a29c5f6/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
new file mode 100644
index 0000000..890021b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
@@ -0,0 +1,21 @@
+#
+# 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.
+#
+
+1=Unsupported operation %1$s in %2$s operator
+2=Error in processing tuple %1$s in a frame