Unify runtime type exceptions by using error code and message template.
-- fixed string_join to be able to handle self-described list;
-- fixed the input arity of day_of_week;
-- added tests for exceptions.
RuntimeDataException is a newly added super class for errors in the data in the runtime.
It has the following subclasses:
-- IncompatibleTypeException, e.g., 1 + "2"
-- InvalidDataFormatException, e.g., boolean(1)
-- OverflowException, e.g., int8(1024)
-- TypeMismatchException, e.g., substr(1, 2)
-- UnderflowException, e.g., int8(-1024)
-- UnsupportedItemTypeException, e.g., string_concat(["a", 1, "c"])
-- UnsupportedTypeException, e.g., rectangle("1,2 3,4") + rectangle("2,5 7,8")
All "root-cause" exceptions thrown from runtime functions have an error code.
Going forward, all "root-cause" exceptions thrown from asterixdb
should have an error code.
Change-Id: Ie4fff8f5e64ffb027910a4899c0246b37ed5bce7
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1313
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: abdullah alamoudi <bamousaa@gmail.com>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntimeFactory.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntimeFactory.java
index 9486a19..767d864 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntimeFactory.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntimeFactory.java
@@ -20,10 +20,10 @@
package org.apache.asterix.algebra.operators.physical;
import org.apache.asterix.common.transactions.JobId;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class CommitRuntimeFactory implements IPushRuntimeFactory {
@@ -56,7 +56,7 @@
}
@Override
- public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException {
if (upsertVarIdx >= 0) {
return new UpsertCommitRuntime(ctx, jobId, datasetId, primaryKeyFields, isTemporaryDatasetWriteJob,
isWriteTransaction, datasetPartitions[ctx.getTaskAttemptId().getTaskId().getPartition()],
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index fec564e..ae31780 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -23,7 +23,6 @@
import java.nio.ByteBuffer;
import java.util.List;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
import org.apache.asterix.common.config.GlobalConfig;
import org.apache.asterix.dataflow.data.common.AqlExpressionTypeComputer;
import org.apache.asterix.dataflow.data.nontagged.AqlMissingWriterFactory;
@@ -43,6 +42,7 @@
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.util.ConstantExpressionUtil;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -214,20 +214,20 @@
IScalarEvaluatorFactory fact = _jobGenCtx.getExpressionRuntimeProvider().createEvaluatorFactory(expr,
_emptyTypeEnv, _emptySchemas, _jobGenCtx);
- IScalarEvaluator eval = fact.createScalarEvaluator(null);
- eval.evaluate(null, p);
- Object t = _emptyTypeEnv.getType(expr);
-
- @SuppressWarnings("rawtypes")
- ISerializerDeserializer serde = _jobGenCtx.getSerializerDeserializerProvider().getSerializerDeserializer(t);
- bbis.setByteBuffer(ByteBuffer.wrap(p.getByteArray(), p.getStartOffset(), p.getLength()), 0);
- IAObject o;
try {
- o = (IAObject) serde.deserialize(dis);
+ IScalarEvaluator eval = fact.createScalarEvaluator(null);
+ eval.evaluate(null, p);
+ Object t = _emptyTypeEnv.getType(expr);
+
+ @SuppressWarnings("rawtypes")
+ ISerializerDeserializer serde = _jobGenCtx.getSerializerDeserializerProvider()
+ .getSerializerDeserializer(t);
+ bbis.setByteBuffer(ByteBuffer.wrap(p.getByteArray(), p.getStartOffset(), p.getLength()), 0);
+ IAObject o = (IAObject) serde.deserialize(dis);
+ return new Pair<>(true, new ConstantExpression(new AsterixConstantValue(o)));
} catch (HyracksDataException e) {
throw new AlgebricksException(e);
}
- return new Pair<Boolean, ILogicalExpression>(true, new ConstantExpression(new AsterixConstantValue(o)));
}
@Override
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
index 06a6f37..4fd8d75 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
@@ -225,7 +225,7 @@
// The user query provides zero parameter for the meta function.
if (variableRequired) {
throw new AlgebricksException("Cannot resolve to ambiguity on the meta function call --"
- + " there are more than once dataset choices!");
+ + " there are more than one dataset choices!");
}
exprRef.setValue(new VariableReferenceExpression(metaVar));
return true;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CompilerExtensionManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CompilerExtensionManager.java
index 4734841..c994abd 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CompilerExtensionManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CompilerExtensionManager.java
@@ -30,6 +30,7 @@
import org.apache.asterix.common.api.IExtension;
import org.apache.asterix.common.config.AsterixExtension;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.compiler.provider.AqlCompilationProvider;
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
@@ -43,10 +44,6 @@
*/
public class CompilerExtensionManager implements IAlgebraExtensionManager {
- private static final String ERROR_MESSAGE_ID_CONFLICT = "Two Extensions share the same Id: %1$s";
- public static final String ERROR_MESSAGE_COMPONENT_CONFLICT =
- "Extension Conflict between %1$s and %2$s both extensions extend %3$s";
-
private final Map<ExtensionId, IExtension> extensions = new HashMap<>();
private final IStatementExecutorExtension statementExecutorExtension;
@@ -75,8 +72,7 @@
IExtension extension = (IExtension) Class.forName(extensionConf.getClassName()).newInstance();
extension.configure(extensionConf.getArgs());
if (extensions.containsKey(extension.getId())) {
- throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.ERROR_EXTENSION_CONFLICT,
- ERROR_MESSAGE_ID_CONFLICT, extension.getId());
+ throw new RuntimeDataException(ErrorCode.ERROR_EXTENSION_ID_CONFLICT, extension.getId());
}
extensions.put(extension.getId(), extension);
switch (extension.getExtensionKind()) {
@@ -101,8 +97,8 @@
private Pair<ExtensionId, ILangCompilationProvider> extendLangCompilationProvider(Language lang,
Pair<ExtensionId, ILangCompilationProvider> cp, ILangExtension le) throws HyracksDataException {
if (cp != null && le.getLangCompilationProvider(lang) != null) {
- throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.ERROR_EXTENSION_CONFLICT,
- ERROR_MESSAGE_COMPONENT_CONFLICT, le.getId(), cp.first, lang.toString());
+ throw new RuntimeDataException(ErrorCode.ERROR_EXTENSION_COMPONENT_CONFLICT, le.getId(), cp.first,
+ lang.toString());
}
return (le.getLangCompilationProvider(lang) != null)
? new Pair<>(le.getId(), le.getLangCompilationProvider(lang)) : cp;
@@ -111,8 +107,7 @@
private IStatementExecutorExtension extendStatementExecutor(IStatementExecutorExtension qte,
IStatementExecutorExtension extension) throws HyracksDataException {
if (qte != null) {
- throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.ERROR_EXTENSION_CONFLICT,
- ERROR_MESSAGE_COMPONENT_CONFLICT, qte.getId(), extension.getId(),
+ throw new RuntimeDataException(ErrorCode.ERROR_EXTENSION_COMPONENT_CONFLICT, qte.getId(), extension.getId(),
IStatementExecutorFactory.class.getSimpleName());
}
return extension;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCExtensionManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCExtensionManager.java
index 759bf09..fcf559f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCExtensionManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCExtensionManager.java
@@ -22,10 +22,10 @@
import java.util.ArrayList;
import java.util.List;
-import org.apache.asterix.app.cc.CompilerExtensionManager;
import org.apache.asterix.common.api.IExtension;
import org.apache.asterix.common.config.AsterixExtension;
import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.metadata.api.IMetadataExtension;
import org.apache.asterix.metadata.entitytupletranslators.MetadataTupleTranslatorProvider;
@@ -76,8 +76,8 @@
private MetadataTupleTranslatorProvider extendTupleTranslator(MetadataTupleTranslatorProvider ttp,
IMetadataExtension tupleTranslatorExtension, IMetadataExtension mde) throws HyracksDataException {
if (ttp != null) {
- throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.ERROR_EXTENSION_CONFLICT,
- CompilerExtensionManager.ERROR_MESSAGE_COMPONENT_CONFLICT, tupleTranslatorExtension.getId(),
+ throw new RuntimeDataException(ErrorCode.ERROR_EXTENSION_COMPONENT_CONFLICT,
+ tupleTranslatorExtension.getId(),
mde.getId(), IMetadataExtension.class.getSimpleName());
}
return mde.getMetadataTupleTranslatorProvider();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionTest.java
new file mode 100644
index 0000000..6a1d6e2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionTest.java
@@ -0,0 +1,115 @@
+
+/*
+ * 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.asterix.runtime;
+
+import static org.mockito.Mockito.mock;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.translator.util.FunctionCollection;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.evaluators.ConstantEvalFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ExceptionTest {
+
+ @Test
+ public void test() throws Exception {
+ List<IFunctionDescriptorFactory> functions = FunctionCollection.getFunctionDescriptorFactories();
+ int testedFunctions = 0;
+ for (IFunctionDescriptorFactory func : functions) {
+ String className = func.getClass().getName();
+ // We test all generated functions except
+ // record and cast functions, which requires type settings.
+ if (className.contains("Gen") && !className.contains("record") && !className.contains("Cast")) {
+ testFunction(func);
+ ++testedFunctions;
+ }
+ }
+ // 208 is the current number of functions with generated code.
+ Assert.assertTrue(testedFunctions >= 208);
+ }
+
+ private void testFunction(IFunctionDescriptorFactory funcFactory) throws Exception {
+ AbstractScalarFunctionDynamicDescriptor funcDesc = (AbstractScalarFunctionDynamicDescriptor) funcFactory
+ .createFunctionDescriptor();
+ int inputArity = funcDesc.getIdentifier().getArity();
+ Iterator<IScalarEvaluatorFactory[]> argEvalFactoryIterator = getArgCombinations(inputArity);
+ while (argEvalFactoryIterator.hasNext()) {
+ IScalarEvaluatorFactory evalFactory = funcDesc.createEvaluatorFactory(argEvalFactoryIterator.next());
+ IHyracksTaskContext ctx = mock(IHyracksTaskContext.class);
+ IScalarEvaluator evaluator = evalFactory.createScalarEvaluator(ctx);
+ IPointable resultPointable = new VoidPointable();
+ try {
+ evaluator.evaluate(null, resultPointable);
+ } catch (Throwable e) {
+ if (e.getMessage() == null) {
+ continue;
+ }
+ if (e.getMessage().startsWith("ASX")) {
+ continue;
+ } else {
+ // Any root-level data exceptions thrown from runtime functions should have an error code.
+ Assert.assertTrue(!(e instanceof HyracksDataException) || (e.getCause() != null));
+ }
+ }
+ }
+ }
+
+ private Iterator<IScalarEvaluatorFactory[]> getArgCombinations(final int inputArity) {
+ final int argSize = inputArity >= 0 ? inputArity : 3;
+ final int numCombinations = (int) Math.pow(ATypeTag.values().length, argSize);
+ return new Iterator<IScalarEvaluatorFactory[]>() {
+ private int index = 0;
+
+ @Override
+ public boolean hasNext() {
+ return index < numCombinations;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory[] next() {
+ IScalarEvaluatorFactory[] scalarEvaluatorFactories = new IScalarEvaluatorFactory[argSize];
+ for (int j = 0; j < argSize; ++j) {
+ int base = (int) Math.pow(ATypeTag.values().length, j);
+ // Enumerates through all possible type tags.
+ byte serializedTypeTag = (byte) ((index / base) % ATypeTag.values().length);
+ scalarEvaluatorFactories[j] = new ConstantEvalFactory(new byte[] { serializedTypeTag });
+ }
+ ++index;
+ return scalarEvaluatorFactories;
+ }
+
+ };
+
+ }
+
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
index d30ffd9..7827297 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
@@ -25,6 +25,7 @@
import java.util.Iterator;
import java.util.List;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -43,19 +44,26 @@
@Test
public void test() throws Exception {
List<IFunctionDescriptorFactory> functions = FunctionCollection.getFunctionDescriptorFactories();
+ int testedFunctions = 0;
for (IFunctionDescriptorFactory func : functions) {
String className = func.getClass().getName();
// We test all generated functions except
- // record functions, which requires type settings (we test them in runtime tests);
- if (className.contains("generated") && !className.contains("record") && !className.contains("Cast")) {
+ // record and cast functions, which requires type settings (we test them in runtime tests).
+ if (className.contains("Gen") && !className.contains("record") && !className.contains("Cast")) {
testFunction(func);
+ ++testedFunctions;
}
}
+ // 208 is the current number of functions with generated code.
+ Assert.assertTrue(testedFunctions >= 208);
}
private void testFunction(IFunctionDescriptorFactory funcFactory) throws Exception {
- AbstractScalarFunctionDynamicDescriptor funcDesc =
- (AbstractScalarFunctionDynamicDescriptor) funcFactory.createFunctionDescriptor();
+ IFunctionDescriptor functionDescriptor = funcFactory.createFunctionDescriptor();
+ if (!(functionDescriptor instanceof AbstractScalarFunctionDynamicDescriptor)) {
+ return;
+ }
+ AbstractScalarFunctionDynamicDescriptor funcDesc = (AbstractScalarFunctionDynamicDescriptor) functionDescriptor;
int inputArity = funcDesc.getIdentifier().getArity();
Iterator<IScalarEvaluatorFactory[]> argEvalFactoryIterator = getArgCombinations(inputArity);
int index = 0;
@@ -76,7 +84,8 @@
}
}
- private Iterator<IScalarEvaluatorFactory[]> getArgCombinations(int argSize) {
+ private Iterator<IScalarEvaluatorFactory[]> getArgCombinations(int inputArity) {
+ int argSize = inputArity >= 0 ? inputArity : 3;
final int numCombinations = 1 << argSize;
return new Iterator<IScalarEvaluatorFactory[]>() {
private int index = 0;
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
index 7b0e015..298c458 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
@@ -423,7 +423,7 @@
<test-case FilePath="exception">
<compilation-unit name="issue_255_create_dataset_error_2">
<output-dir compare="Text">none</output-dir>
- <expected-error>Error: The partitioning key [open-type] cannot be of type RECORD.</expected-error>
+ <expected-error>Error: The partitioning key [open-type] cannot be of type record.</expected-error>
</compilation-unit>
</test-case>
<!-- Feed datasets are not supported anymore
@@ -449,37 +449,37 @@
<test-case FilePath="exception">
<compilation-unit name="issue_384_create_index_error_1">
<output-dir compare="Text">none</output-dir>
- <expected-error>Error: The field "[loc]" which is of type POINT cannot be indexed using the BTree index.</expected-error>
+ <expected-error>Error: The field "[loc]" which is of type point cannot be indexed using the BTree index.</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_384_create_index_error_2">
<output-dir compare="Text">none</output-dir>
- <expected-error>Error: The field "[age]" which is of type INT32 cannot be indexed using the RTree index.</expected-error>
+ <expected-error>Error: The field "[age]" which is of type int32 cannot be indexed using the RTree index.</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_384_create_index_error_3">
<output-dir compare="Text">none</output-dir>
- <expected-error>Error: The field "[loc]" which is of type POINT cannot be indexed using the Length Partitioned Keyword index.</expected-error>
+ <expected-error>Error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index.</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_384_create_index_error_4">
<output-dir compare="Text">none</output-dir>
- <expected-error>Error: The field "[loc]" which is of type POINT cannot be indexed using the Length Partitioned Keyword index.</expected-error>
+ <expected-error>Error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index.</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_384_create_index_error_5">
<output-dir compare="Text">none</output-dir>
- <expected-error>Error: The field "[loc]" which is of type POINT cannot be indexed using the Length Partitioned N-Gram index.</expected-error>
+ <expected-error>Error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index.</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_384_create_index_error_6">
<output-dir compare="Text">none</output-dir>
- <expected-error>Error: The field "[loc]" which is of type POINT cannot be indexed using the Length Partitioned N-Gram index.</expected-error>
+ <expected-error>Error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index.</expected-error>
</compilation-unit>
</test-case>
</test-group>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/comparison/ComparisonQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries/comparison/ComparisonQueries.xml
index 8493df2..e549a92 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/comparison/ComparisonQueries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/comparison/ComparisonQueries.xml
@@ -172,43 +172,43 @@
<test-case FilePath="comparison">
<compilation-unit name="issue363_inequality_duration">
<output-dir compare="Text">issue363_inequality_duration</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Comparison operations (GT, GE, LT, and LE) for the DURATION type are not defined</expected-error>
+ <expected-error>Unsupported type: comparison operations (>, >=, <, and <=) cannot process input type duration</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="comparison">
<compilation-unit name="issue363_inequality_interval">
<output-dir compare="Text">issue363_inequality_interval</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Comparison operations (GT, GE, LT, and LE) for the INTERVAL type are not defined</expected-error>
+ <expected-error>Unsupported type: comparison operations (>, >=, <, and <=) cannot process input type interval</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="comparison">
<compilation-unit name="issue363_inequality_point">
<output-dir compare="Text">issue363_inequality_point</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Comparison operations (GT, GE, LT, and LE) for the POINT type are not defined</expected-error>
+ <expected-error>Unsupported type: comparison operations (>, >=, <, and <=) cannot process input type point</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="comparison">
<compilation-unit name="issue363_inequality_line">
<output-dir compare="Text">issue363_inequality_line</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Comparison operations (GT, GE, LT, and LE) for the LINE type are not defined</expected-error>
+ <expected-error>Unsupported type: comparison operations (>, >=, <, and <=) cannot process input type line</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="comparison">
<compilation-unit name="issue363_inequality_polygon">
<output-dir compare="Text">issue363_inequality_polygon</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Comparison operations (GT, GE, LT, and LE) for the POLYGON type are not defined</expected-error>
+ <expected-error>Unsupported type: comparison operations (>, >=, <, and <=) cannot process input type polygon</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="comparison">
<compilation-unit name="issue363_inequality_rectangle">
<output-dir compare="Text">issue363_inequality_rectangle</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Comparison operations (GT, GE, LT, and LE) for the RECTANGLE type are not defined</expected-error>
+ <expected-error>Unsupported type: comparison operations (>, >=, <, and <=) cannot process input type rectangle</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="comparison">
<compilation-unit name="issue363_inequality_circle">
<output-dir compare="Text">issue363_inequality_circle</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Comparison operations (GT, GE, LT, and LE) for the CIRCLE type are not defined</expected-error>
+ <expected-error>Unsupported type: comparison operations (>, >=, <, and <=) cannot process input type circle</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="comparison">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index 942b546..dd859c4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -452,19 +452,19 @@
<test-case FilePath="aggregate">
<compilation-unit name="avg_mixed">
<output-dir compare="Text">avg_mixed</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Unexpected type STRING</expected-error>
+ <expected-error>Type incompatibility: function agg-avg gets incompatible input values: string and float</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="aggregate">
<compilation-unit name="sum_mixed">
<output-dir compare="Text">sum_mixed</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Unexpected type STRING</expected-error>
+ <expected-error>Type incompatibility: function agg-sum gets incompatible input values: string and float</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="aggregate">
<compilation-unit name="min_mixed">
<output-dir compare="Text">min_mixed</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Unexpected type STRING</expected-error>
+ <expected-error>Type incompatibility: function min/max gets incompatible input values: string and float</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="aggregate">
@@ -913,19 +913,19 @@
<test-case FilePath="aggregate-sql">
<compilation-unit name="avg_mixed">
<output-dir compare="Text">avg_mixed</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Unexpected type STRING</expected-error>
+ <expected-error>Type incompatibility: function agg-avg gets incompatible input values: string and float</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="aggregate-sql">
<compilation-unit name="sum_mixed">
<output-dir compare="Text">sum_mixed</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Unexpected type STRING</expected-error>
+ <expected-error>Type incompatibility: function agg-sum gets incompatible input values: string and float</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="aggregate-sql">
<compilation-unit name="min_mixed">
<output-dir compare="Text">min_mixed</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Unexpected type STRING</expected-error>
+ <expected-error>Type incompatibility: function min/max gets incompatible input values: string and float</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="aggregate-sql">
@@ -1560,7 +1560,7 @@
<test-case FilePath="dml">
<compilation-unit name="insert-duplicated-keys-from-query">
<output-dir compare="Text">insert-duplicated-keys-from-query</output-dir>
- <expected-error>org.apache.hyracks.storage.am.common.exceptions.TreeIndexDuplicateKeyException: Failed to insert key since key already exists</expected-error>
+ <expected-error>Failed to insert key since key already exists</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="dml">
@@ -1755,7 +1755,7 @@
<test-case FilePath="dml">
<compilation-unit name="insert-with-autogenerated-pk_adm_02">
<output-dir compare="Text">insert-with-autogenerated-pk_adm_02</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Duplicate field id encountered</expected-error>
+ <expected-error>Duplicate field id encountered</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="dml">
@@ -2566,7 +2566,7 @@
<test-case FilePath="fuzzyjoin">
<compilation-unit name="dblp-aqlplus_2">
<output-dir compare="Text">dblp-aqlplus_2</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Invalid types STRING given as arguments to jaccard</expected-error>
+ <expected-error>Type mismatch: function similarity-jaccard expects its 1st input parameter to be type orderedlist or unorderedlist, but the actual input type is string</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="fuzzyjoin">
@@ -3215,7 +3215,7 @@
<test-case FilePath="meta">
<compilation-unit name="query_dataset_with_meta_failure">
<output-dir compare="Text">query_dataset_with_meta_failure</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Cannot resolve to ambiguity on the meta function call -- there are more than once dataset choices!</expected-error>
+ <expected-error>Cannot resolve to ambiguity on the meta function call -- there are more than one dataset choices!</expected-error>
</compilation-unit>
</test-case>
</test-group>
@@ -3224,8 +3224,8 @@
<compilation-unit name="partition-by-nonexistent-field"> <!-- Seriously?? 3 expected errors -->
<output-dir compare="Text">partition-by-nonexistent-field</output-dir>
<expected-error>Type not found for partitioning key [id]</expected-error>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Cannot find dataset</expected-error>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Could not find dataset</expected-error>
+ <expected-error>Cannot find dataset</expected-error>
+ <expected-error>Could not find dataset</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="misc">
@@ -3338,25 +3338,25 @@
<test-case FilePath="open-index-enforced/error-checking">
<compilation-unit name="index-on-closed-type">
<output-dir compare="Text">index-on-closed-type</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Typed index on "[value]" field could be created only for open datatype</expected-error>
+ <expected-error>Typed index on "[value]" field could be created only for open datatype</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="open-index-enforced/error-checking">
<compilation-unit name="index-type-collision">
<output-dir compare="Text">index-type-collision</output-dir>
- <expected-error>org.apache.asterix.common.exceptions.AsterixException: Cannot create index testIdx2 , enforced index testIdx1 on field "[value]" is already defined with type "[INT32]"</expected-error>
+ <expected-error>org.apache.asterix.common.exceptions.AsterixException: Cannot create index testIdx2 , enforced index testIdx1 on field "[value]" is already defined with type "[int32]"</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="open-index-enforced/error-checking">
<compilation-unit name="index-type-promotion-collision">
<output-dir compare="Text">index-type-promotion-collision</output-dir>
- <expected-error>org.apache.asterix.common.exceptions.AsterixException: Cannot create index testIdx2 , enforced index testIdx1 on field "[value]" is already defined with type "[INT64]"</expected-error>
+ <expected-error>org.apache.asterix.common.exceptions.AsterixException: Cannot create index testIdx2 , enforced index testIdx1 on field "[value]" is already defined with type "[int64]"</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="open-index-enforced/error-checking">
<compilation-unit name="missing-enforce-statement">
<output-dir compare="Text">missing-enforce-statement</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Cannot create typed index on "[value]" field without enforcing it's type</expected-error>
+ <expected-error>Cannot create typed index on "[value]" field without enforcing it's type</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="open-index-enforced/error-checking">
@@ -3368,7 +3368,7 @@
<test-case FilePath="open-index-enforced/error-checking">
<compilation-unit name="object-type-collision">
<output-dir compare="Text">object-type-collision</output-dir>
- <expected-error>org.apache.asterix.common.exceptions.AsterixException: A field "[value]" is already defined with the type "STRING"</expected-error>
+ <expected-error>org.apache.asterix.common.exceptions.AsterixException: A field "[value]" is already defined with the type "string"</expected-error>
</compilation-unit>
</test-case>
</test-group>
@@ -4689,7 +4689,7 @@
<test-case FilePath="open-closed"><!-- Throws two exceptions. need to be checked. proposal: (fixed expected results) -->
<compilation-unit name="query-issue410">
<output-dir compare="Text">query-issue410</output-dir>
- <expected-error>HyracksDataException: ASX0000: Field type DOUBLE can't be promoted to type STRING</expected-error>
+ <expected-error>Field type double can't be promoted to type string</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="open-closed">
@@ -6589,7 +6589,7 @@
<test-case FilePath="load">
<compilation-unit name="issue650_query">
<output-dir compare="Text">none</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Unable to load dataset Users since it does not exist</expected-error>
+ <expected-error>Unable to load dataset Users since it does not exist</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="load">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index f89fd63..1030454 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -132,19 +132,19 @@
<test-case FilePath="aggregate">
<compilation-unit name="avg_mixed">
<output-dir compare="Text">avg_mixed</output-dir>
- <expected-error>Unexpected type STRING in aggregation input stream. Expected type FLOAT</expected-error>
+ <expected-error>Type incompatibility: function agg-avg gets incompatible input values: string and float</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="aggregate">
<compilation-unit name="sum_mixed">
<output-dir compare="Text">sum_mixed</output-dir>
- <expected-error>Unexpected type STRING in aggregation input stream. Expected type (or a promotable type to)FLOAT</expected-error>
+ <expected-error>Type incompatibility: function agg-sum gets incompatible input values: string and float</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="aggregate">
<compilation-unit name="min_mixed">
<output-dir compare="Text">min_mixed</output-dir>
- <expected-error>Unexpected type STRING in aggregation input stream. Expected type FLOAT</expected-error>
+ <expected-error>Type incompatibility: function min/max gets incompatible input values: string and float</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="aggregate">
@@ -568,19 +568,19 @@
<test-case FilePath="aggregate-sql">
<compilation-unit name="avg_mixed">
<output-dir compare="Text">avg_mixed</output-dir>
- <expected-error>Unexpected type STRING in aggregation input stream. Expected type FLOAT</expected-error>
+ <expected-error>Type incompatibility: function agg-avg gets incompatible input values: string and float</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="aggregate-sql">
<compilation-unit name="sum_mixed">
<output-dir compare="Text">sum_mixed</output-dir>
- <expected-error>Unexpected type STRING in aggregation input stream. Expected type (or a promotable type to)FLOAT</expected-error>
+ <expected-error>Type incompatibility: function agg-sum gets incompatible input values: string and float</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="aggregate-sql">
<compilation-unit name="min_mixed">
<output-dir compare="Text">min_mixed</output-dir>
- <expected-error>Unexpected type STRING in aggregation input stream. Expected type FLOAT</expected-error>
+ <expected-error>Type incompatibility: function min/max gets incompatible input values: string and float</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="aggregate-sql">
@@ -1112,43 +1112,43 @@
<test-case FilePath="comparison">
<compilation-unit name="issue363_inequality_duration">
<output-dir compare="Text">issue363_inequality_duration</output-dir>
- <expected-error>Comparison operations (GT, GE, LT, and LE) for the DURATION type are not defined</expected-error>
+ <expected-error>Unsupported type: comparison operations (>, >=, <, and <=) cannot process input type duration</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="comparison">
<compilation-unit name="issue363_inequality_interval">
<output-dir compare="Text">issue363_inequality_interval</output-dir>
- <expected-error>Comparison operations (GT, GE, LT, and LE) for the INTERVAL type are not defined</expected-error>
+ <expected-error>Unsupported type: comparison operations (>, >=, <, and <=) cannot process input type interval</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="comparison">
<compilation-unit name="issue363_inequality_point">
<output-dir compare="Text">issue363_inequality_point</output-dir>
- <expected-error>Comparison operations (GT, GE, LT, and LE) for the POINT type are not defined</expected-error>
+ <expected-error>Unsupported type: comparison operations (>, >=, <, and <=) cannot process input type point</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="comparison">
<compilation-unit name="issue363_inequality_line">
<output-dir compare="Text">issue363_inequality_line</output-dir>
- <expected-error>Comparison operations (GT, GE, LT, and LE) for the LINE type are not defined</expected-error>
+ <expected-error>Unsupported type: comparison operations (>, >=, <, and <=) cannot process input type line</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="comparison">
<compilation-unit name="issue363_inequality_polygon">
<output-dir compare="Text">issue363_inequality_polygon</output-dir>
- <expected-error>Comparison operations (GT, GE, LT, and LE) for the POLYGON type are not defined</expected-error>
+ <expected-error>Unsupported type: comparison operations (>, >=, <, and <=) cannot process input type polygon</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="comparison">
<compilation-unit name="issue363_inequality_rectangle">
<output-dir compare="Text">issue363_inequality_rectangle</output-dir>
- <expected-error>Comparison operations (GT, GE, LT, and LE) for the RECTANGLE type are not defined</expected-error>
+ <expected-error>Unsupported type: comparison operations (>, >=, <, and <=) cannot process input type rectangle</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="comparison">
<compilation-unit name="issue363_inequality_circle">
<output-dir compare="Text">issue363_inequality_circle</output-dir>
- <expected-error>Comparison operations (GT, GE, LT, and LE) for the CIRCLE type are not defined</expected-error>
+ <expected-error>Unsupported type: comparison operations (>, >=, <, and <=) cannot process input type circle</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="comparison">
@@ -2440,7 +2440,7 @@
<test-case FilePath="global-aggregate">
<compilation-unit name="q06_error">
<output-dir compare="Text">q01</output-dir>
- <expected-error>Unsupported type: STRING</expected-error>
+ <expected-error>Type mismatch: function scan-collection expects its 1st input parameter to be type orderedlist or unorderedlist, but the actual input type is string</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="global-aggregate">
@@ -3171,7 +3171,7 @@
<test-case FilePath="open-index-enforced/error-checking">
<compilation-unit name="enforced-field-type-collision">
<output-dir compare="Text">enforced-field-type-collision</output-dir>
- <expected-error>A field "[value]" is already defined with the type "STRING"</expected-error>
+ <expected-error>A field "[value]" is already defined with the type "string"</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="open-index-enforced/error-checking">
@@ -4640,7 +4640,7 @@
<test-case FilePath="open-closed">
<compilation-unit name="query-issue410">
<output-dir compare="Text">query-issue410</output-dir>
- <expected-error>ASX0000: Field type DOUBLE can't be promoted to type STRING</expected-error>
+ <expected-error>Field type double can't be promoted to type string</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="open-closed">
@@ -5433,7 +5433,7 @@
<test-case FilePath="string">
<compilation-unit name="repeat_error">
<output-dir compare="Text">repeat</output-dir>
- <expected-error>repeat: expects a non-negative repeating number but got -1</expected-error>
+ <expected-error>Invalid value: function asterix:repeat expects its 1 input parameter to be a non-negative value, but gets -1</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="string">
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 a56d6f6..b182e35 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,12 +18,91 @@
*/
package org.apache.asterix.common.exceptions;
+import java.util.HashMap;
+import java.util.Map;
+
+// Error code:
+// 0 --- 999: runtime errors
+// 1000 ---- 1999: compilation errors
+// 2000 ---- 2999: storage errors
+// 3000 ---- 3999: feed errors
+// 4000 ---- 4999: lifecycle management errors
public class ErrorCode {
public static final String ASTERIX = "ASX";
- public static final int ERROR_CASTING_FIELD = 0;
- public static final int ERROR_EXTENSION_CONFLICT = 1;
- public static final int ERROR_PARSE_ERROR = 2;
+
+ // Extension errors
+ public static final int ERROR_EXTENSION_ID_CONFLICT = 4001;
+ public static final int ERROR_EXTENSION_COMPONENT_CONFLICT = 4002;
+
+ // Runtime errors
+ public static final int ERROR_CASTING_FIELD = 1;
+ public static final int ERROR_TYPE_MISMATCH = 2;
+ public static final int ERROR_TYPE_INCOMPATIBLE = 3;
+ public static final int ERROR_TYPE_UNSUPPORTED = 4;
+ public static final int ERROR_TYPE_ITEM = 5;
+ public static final int ERROR_INVALID_FORMAT = 6;
+ public static final int ERROR_OVERFLOW = 7;
+ public static final int ERROR_UNDERFLOW = 8;
+ public static final int ERROR_INJECTED_FAILURE = 9;
+ public static final int ERROR_NEGATIVE_VALUE = 10;
+ public static final int ERROR_OUT_OF_BOUND = 11;
+ public static final int ERROR_COERCION = 12;
+ public static final int ERROR_DUPLICATE_FIELD = 13;
+
+ // Compilation errors
+ public static final int ERROR_PARSE_ERROR = 1001;
+
+ 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)";
+ public static final String ERROR_MESSAGE_INVALID_FORMAT = "Invalid format for %1$s in %2$s";
+ public static final String ERROR_MESSAGE_OVERFLOW = "Overflow happend in %1$s";
+ public static final String ERROR_MESSAGE_UNDERFLOW = "Underflow happend in %1$s";
+ public static final String ERROR_MESSAGE_INJECTED_FAILURE = "Injected failure in %1$s";
+ public 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";
+ public static final String ERROR_MESSAGE_OUT_OF_BOUND = "Index out of bound in %1$s: %2$s";
+ public static final String ERROR_MESSAGE_COERCION = "Invalid implicit scalar to collection coercion in %1$s";
+ public static final String ERROR_MESSAGE_DUPLICATE_FIELD = "Get duplicate fields in %1$s";
+
+ private static Map<Integer, String> errorMessageMap = new HashMap<>();
+
+ static {
+ // compilation 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, ERROR_MESSAGE_DUPLICATE_FIELD);
+
+ // lifecycle management errors
+ errorMessageMap.put(ERROR_EXTENSION_ID_CONFLICT, ERROR_MESSAGE_ID_CONFLICT);
+ errorMessageMap.put(ERROR_EXTENSION_COMPONENT_CONFLICT, ERROR_MESSAGE_COMPONENT_CONFLICT);
+ }
private ErrorCode() {
}
+
+ public static String getErrorMessage(int errorCode) {
+ String msg = errorMessageMap.get(errorCode);
+ if (msg == null) {
+ throw new IllegalStateException("Undefined error code: " + errorCode);
+ }
+ return msg;
+ }
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.java
new file mode 100644
index 0000000..fded0d9
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.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.asterix.common.exceptions;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class RuntimeDataException extends HyracksDataException {
+
+ public RuntimeDataException(int errorCode, Serializable... params) {
+ super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), params);
+ }
+
+ public RuntimeDataException(int errorCode, Throwable cause, Serializable... params) {
+ super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), params);
+ addSuppressed(cause);
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java
index 789e9ba..3e47f36 100755
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java
@@ -37,6 +37,7 @@
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -55,7 +56,7 @@
protected final JavaFunctionHelper functionHelper;
public ExternalFunction(IExternalFunctionInfo finfo, IScalarEvaluatorFactory args[], IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
this.finfo = finfo;
this.evaluatorFactories = args;
argumentEvaluators = new IScalarEvaluator[args.length];
@@ -85,7 +86,7 @@
externalFunctionFactory = (IFunctionFactory) clazz.newInstance();
externalFunction = externalFunctionFactory.getExternalFunction();
} catch (Exception e) {
- throw new AlgebricksException(" Unable to load/instantiate class " + classname, e);
+ throw new HyracksDataException(" Unable to load/instantiate class " + classname, e);
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunctionProvider.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunctionProvider.java
index 4f5242e..b344598 100755
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunctionProvider.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunctionProvider.java
@@ -23,17 +23,17 @@
import org.apache.asterix.external.api.IFunctionHelper;
import org.apache.asterix.om.functions.IExternalFunctionInfo;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
public class ExternalFunctionProvider {
public static IExternalFunction getExternalFunctionEvaluator(IExternalFunctionInfo finfo,
- IScalarEvaluatorFactory args[], IHyracksTaskContext context) throws AlgebricksException {
+ IScalarEvaluatorFactory args[], IHyracksTaskContext context) throws HyracksDataException {
switch (finfo.getKind()) {
case SCALAR:
return new ExternalScalarFunction(finfo, args, context);
@@ -49,17 +49,17 @@
class ExternalScalarFunction extends ExternalFunction implements IExternalScalarFunction, IScalarEvaluator {
public ExternalScalarFunction(IExternalFunctionInfo finfo, IScalarEvaluatorFactory args[],
- IHyracksTaskContext context) throws AlgebricksException {
+ IHyracksTaskContext context) throws HyracksDataException {
super(finfo, args, context);
try {
initialize(functionHelper);
} catch (Exception e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
setArguments(tuple);
evaluate(functionHelper);
@@ -67,22 +67,26 @@
functionHelper.reset();
} catch (Exception e) {
e.printStackTrace();
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
@Override
- public void evaluate(IFunctionHelper argumentProvider) throws Exception {
- resultBuffer.reset();
- ((IExternalScalarFunction) externalFunction).evaluate(argumentProvider);
+ public void evaluate(IFunctionHelper argumentProvider) throws HyracksDataException {
+ try {
+ resultBuffer.reset();
+ ((IExternalScalarFunction) externalFunction).evaluate(argumentProvider);
/*
* Make sure that if "setResult" is not called,
* or the result object is missing we let Hyracks storage manager know
* we want to discard a missing object
*/
- byte byteOutput = resultBuffer.getByteArray()[0];
- if (!argumentProvider.isValidResult() || byteOutput == ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
- resultBuffer.getDataOutput().writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
+ byte byteOutput = resultBuffer.getByteArray()[0];
+ if (!argumentProvider.isValidResult() || byteOutput == ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
+ resultBuffer.getDataOutput().writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
+ }
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarFunctionEvaluatorFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarFunctionEvaluatorFactory.java
index 7c1ce4b..99d40c7 100755
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarFunctionEvaluatorFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarFunctionEvaluatorFactory.java
@@ -23,6 +23,7 @@
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class ExternalScalarFunctionEvaluatorFactory implements IScalarEvaluatorFactory {
@@ -37,7 +38,7 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return (ExternalScalarFunction) ExternalFunctionProvider.getExternalFunctionEvaluator(finfo, args, ctx);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java
index 50dc28e..912192d 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java
@@ -59,7 +59,7 @@
private boolean isValidResult = false;
public JavaFunctionHelper(IExternalFunctionInfo finfo, IDataOutputProvider outputProvider)
- throws AlgebricksException {
+ throws HyracksDataException {
this.finfo = finfo;
this.outputProvider = outputProvider;
this.pointableVisitor = new JObjectPointableVisitor();
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ClassAd.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ClassAd.java
index 4f27cd8..7ac637f 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ClassAd.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ClassAd.java
@@ -478,7 +478,7 @@
ExprTree tree = pRef;
// sanity checks
if (attrName.isEmpty() || pRef == null) {
- throw new HyracksDataException();
+ throw new HyracksDataException("Attribute name is empty");
}
if (tree != null) {
CaseInsensitiveString pstrAttr = objectPool.caseInsensitiveStringPool.get();
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java
index 4460fd9..ad8375cf 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java
@@ -55,7 +55,7 @@
out.writeDouble(instance.getP().getY());
out.writeDouble(instance.getRadius());
} catch (IOException e) {
- throw new HyracksDataException();
+ throw new HyracksDataException(e);
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
index c2e7729..3e6bacd 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
@@ -49,7 +49,7 @@
try {
out.writeLong(instance.getMilliseconds());
} catch (IOException e) {
- throw new HyracksDataException();
+ throw new HyracksDataException(e);
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java
index aef8de0..0ffa0d0 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java
@@ -56,7 +56,7 @@
out.writeDouble(instance.getP2().getX());
out.writeDouble(instance.getP2().getY());
} catch (IOException e) {
- throw new HyracksDataException();
+ throw new HyracksDataException(e);
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
index 7e0bf0f..b12b964 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
@@ -56,7 +56,7 @@
out.writeDouble(instance.getP2().getX());
out.writeDouble(instance.getP2().getY());
} catch (IOException e) {
- throw new HyracksDataException();
+ throw new HyracksDataException(e);
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
index eb6942f..f927204 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
@@ -49,7 +49,7 @@
try {
out.writeInt(instance.getMonths());
} catch (IOException e) {
- throw new HyracksDataException();
+ throw new HyracksDataException(e);
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlBinaryIntegerInspector.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlBinaryIntegerInspector.java
index 21877f5..d7b3845 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlBinaryIntegerInspector.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlBinaryIntegerInspector.java
@@ -40,6 +40,6 @@
@Override
public int getIntegerValue(byte[] bytes, int offset, int length) throws HyracksDataException {
- return ATypeHierarchy.getIntegerValue(bytes, offset);
+ return ATypeHierarchy.getIntegerValue("integer-inspector", 0, bytes, offset);
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/AsterixTemporalTypeParseException.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/AsterixTemporalTypeParseException.java
index ba7dbda..78e64c9 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/AsterixTemporalTypeParseException.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/AsterixTemporalTypeParseException.java
@@ -24,14 +24,6 @@
private static final long serialVersionUID = 1L;
- public AsterixTemporalTypeParseException() {
- super();
- }
-
- public AsterixTemporalTypeParseException(Exception ex) {
- super(ex);
- }
-
public AsterixTemporalTypeParseException(String msg) {
super(msg);
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AsterixBuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AsterixBuiltinFunctions.java
index 66581c4..29a693c 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AsterixBuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AsterixBuiltinFunctions.java
@@ -726,7 +726,7 @@
public static final FunctionIdentifier ADJUST_DATETIME_FOR_TIMEZONE = new FunctionIdentifier(
FunctionConstants.ASTERIX_NS, "adjust-datetime-for-timezone", 2);
public static final FunctionIdentifier DAY_OF_WEEK = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
- "day-of-week");
+ "day-of-week", 1);
public static final FunctionIdentifier PARSE_DATE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
"parse-date", 2);
public static final FunctionIdentifier PARSE_TIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ATypeTag.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ATypeTag.java
index 4d6a96c..ec1805f 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ATypeTag.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ATypeTag.java
@@ -98,6 +98,8 @@
public static final byte SERIALIZED_INT64_TYPE_TAG = INT64.serialize();
public static final byte SERIALIZED_FLOAT_TYPE_TAG = FLOAT.serialize();
public static final byte SERIALIZED_BINARY_TYPE_TAG = BINARY.serialize();
+ public static final byte SERIALIZED_UUID_TYPE_TAG = UUID.serialize();
+
/*
* Serialized Tags end
*/
@@ -131,4 +133,9 @@
|| this == ATypeTag.UNION;
}
+ @Override
+ public String toString() {
+ return this.name().toLowerCase();
+ }
+
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ATypeHierarchy.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ATypeHierarchy.java
index af4d2a2..515ea67 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ATypeHierarchy.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ATypeHierarchy.java
@@ -25,6 +25,8 @@
import java.util.Map;
import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.om.base.ADouble;
import org.apache.asterix.om.base.AFloat;
import org.apache.asterix.om.base.AInt16;
@@ -181,11 +183,11 @@
ATypeTag targetTypeTag) throws AlgebricksException {
ATypeTag sourceTypeTag = sourceObject.getType().getTypeTag();
AsterixConstantValue asterixNewConstantValue = null;
- short tmpShortValue = 0;
- int tmpIntValue = 0;
- long tmpLongValue = 0;
- float tmpFloatValue = 0.0f;
- double tmpDoubleValue = 0.0;
+ short tmpShortValue;
+ int tmpIntValue;
+ long tmpLongValue;
+ float tmpFloatValue;
+ double tmpDoubleValue;
// if the constant type and target type does not match, we do a type conversion
if (sourceTypeTag != targetTypeTag) {
@@ -645,18 +647,21 @@
}
+
// Get an INT value from numeric types array. We assume the first byte contains the type tag.
- public static int getIntegerValue(byte[] bytes, int offset) throws HyracksDataException {
- return getIntegerValueWithDifferentTypeTagPosition(bytes, offset + 1, offset);
+ public static int getIntegerValue(String name, int argIndex, byte[] bytes, int offset) throws HyracksDataException {
+ return getIntegerValueWithDifferentTypeTagPosition(name, argIndex, bytes, offset + 1, offset);
}
// Get an INT value from numeric types array. We assume the specific location of a byte array contains the type tag.
- public static int getIntegerValueWithDifferentTypeTagPosition(byte[] bytes, int offset, int typeTagPosition)
- throws HyracksDataException {
- int value = 0;
-
+ public static int getIntegerValueWithDifferentTypeTagPosition(String name, int argIndex, byte[] bytes, int offset,
+ int typeTagPosition) throws HyracksDataException {
+ int value;
ATypeTag sourceTypeTag = ATypeTag.VALUE_TYPE_MAPPING[bytes[typeTagPosition]];
+ if (sourceTypeTag == null) {
+ throw new RuntimeDataException(ErrorCode.ERROR_INVALID_FORMAT, name, argIndex);
+ }
switch (sourceTypeTag) {
case INT64:
value = (int) LongPointable.getLong(bytes, offset);
@@ -677,26 +682,27 @@
value = (int) DoublePointable.getDouble(bytes, offset);
break;
default:
- throw new HyracksDataException(
- "Type casting error while getting an INT32 value: expected INT8/16/32/64/FLOAT/DOUBLE but got "
- + sourceTypeTag + ".");
+ throw new RuntimeDataException(ErrorCode.ERROR_TYPE_MISMATCH, name, argIndex, sourceTypeTag,
+ ATypeTag.INT8, ATypeTag.INT16, ATypeTag.INT32, ATypeTag.INT64, ATypeTag.FLOAT, ATypeTag.DOUBLE);
+
}
return value;
}
// Get a LONG (INT64) value from numeric types array. We assume the first byte contains the type tag.
- public static long getLongValue(byte[] bytes, int offset) throws HyracksDataException {
- return getLongValueWithDifferentTypeTagPosition(bytes, offset + 1, offset);
+ public static long getLongValue(String name, int argIndex, byte[] bytes, int offset) throws HyracksDataException {
+ return getLongValueWithDifferentTypeTagPosition(name, argIndex, bytes, offset + 1, offset);
}
// Get a LONG (INT64) value from numeric types array. We assume the specific location of a byte array contains the type tag.
- public static long getLongValueWithDifferentTypeTagPosition(byte[] bytes, int offset, int typeTagPosition)
- throws HyracksDataException {
- long value = 0;
-
+ private static long getLongValueWithDifferentTypeTagPosition(String name, int argIndex, byte[] bytes, int offset,
+ int typeTagPosition) throws HyracksDataException {
+ long value;
ATypeTag sourceTypeTag = ATypeTag.VALUE_TYPE_MAPPING[bytes[typeTagPosition]];
-
+ if (sourceTypeTag == null) {
+ throw new RuntimeDataException(ErrorCode.ERROR_INVALID_FORMAT, name, argIndex);
+ }
switch (sourceTypeTag) {
case INT64:
value = LongPointable.getLong(bytes, offset);
@@ -717,66 +723,28 @@
value = (long) DoublePointable.getDouble(bytes, offset);
break;
default:
- throw new HyracksDataException(
- "Type casting error while getting an INT64 value: expected INT8/16/32/64/FLOAT/DOUBLE but got "
- + sourceTypeTag + ".");
- }
-
- return value;
- }
-
- // Get a FLOAT value from numeric types array. We assume the first byte contains the type tag.
- public static float getFloatValue(byte[] bytes, int offset) throws HyracksDataException {
- return getFloatValueWithDifferentTypeTagPosition(bytes, offset + 1, offset);
- }
-
- // Get a FLOAT value from numeric types array. We assume the specific location of a byte array contains the type tag.
- public static float getFloatValueWithDifferentTypeTagPosition(byte[] bytes, int offset, int typeTagPosition)
- throws HyracksDataException {
- float value = 0;
-
- ATypeTag sourceTypeTag = ATypeTag.VALUE_TYPE_MAPPING[bytes[typeTagPosition]];
-
- switch (sourceTypeTag) {
- case INT64:
- value = LongPointable.getLong(bytes, offset);
- break;
- case INT32:
- value = IntegerPointable.getInteger(bytes, offset);
- break;
- case INT8:
- value = bytes[offset];
- break;
- case INT16:
- value = ShortPointable.getShort(bytes, offset);
- break;
- case FLOAT:
- value = FloatPointable.getFloat(bytes, offset);
- break;
- case DOUBLE:
- value = (float) DoublePointable.getDouble(bytes, offset);
- break;
- default:
- throw new HyracksDataException(
- "Type casting error while getting a FLOAT value: expected INT8/16/32/64/FLOAT/DOUBLE but got "
- + sourceTypeTag + ".");
+ throw new RuntimeDataException(ErrorCode.ERROR_TYPE_MISMATCH, name, argIndex, sourceTypeTag,
+ ATypeTag.INT8, ATypeTag.INT16, ATypeTag.INT32, ATypeTag.INT64, ATypeTag.FLOAT, ATypeTag.DOUBLE);
}
return value;
}
// Get a DOUBLE value from numeric types array. We assume the first byte contains the type tag.
- public static double getDoubleValue(byte[] bytes, int offset) throws HyracksDataException {
- return getDoubleValueWithDifferentTypeTagPosition(bytes, offset + 1, offset);
+ public static double getDoubleValue(String name, int argIndex, byte[] bytes, int offset)
+ throws HyracksDataException {
+ return getDoubleValueWithDifferentTypeTagPosition(name, argIndex, bytes, offset + 1, offset);
}
// Get a DOUBLE value from numeric types array. We assume the specific location of a byte array contains the type tag.
- public static double getDoubleValueWithDifferentTypeTagPosition(byte[] bytes, int offset, int typeTagPosition)
+ private static double getDoubleValueWithDifferentTypeTagPosition(String name, int argIndex, byte[] bytes,
+ int offset, int typeTagPosition)
throws HyracksDataException {
- double value = 0;
-
+ double value;
ATypeTag sourceTypeTag = ATypeTag.VALUE_TYPE_MAPPING[bytes[typeTagPosition]];
-
+ if (sourceTypeTag == null) {
+ throw new RuntimeDataException(ErrorCode.ERROR_INVALID_FORMAT, name, argIndex);
+ }
switch (sourceTypeTag) {
case INT64:
value = LongPointable.getLong(bytes, offset);
@@ -797,9 +765,8 @@
value = DoublePointable.getDouble(bytes, offset);
break;
default:
- throw new HyracksDataException(
- "Type casting error while getting a DOUBLE value: expected INT8/16/32/64/FLOAT/DOUBLE but got "
- + sourceTypeTag + ".");
+ throw new RuntimeDataException(ErrorCode.ERROR_TYPE_MISMATCH, name, argIndex, sourceTypeTag,
+ ATypeTag.INT8, ATypeTag.INT16, ATypeTag.INT32, ATypeTag.INT64, ATypeTag.FLOAT, ATypeTag.DOUBLE);
}
return value;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementAggregateDescriptor.java
index 2a94418..1aef0eb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementAggregateDescriptor.java
@@ -21,7 +21,7 @@
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -38,8 +38,7 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new FirstElementEvalFactory(args, false);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java
index fe3a7ec..bea8d5f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java
@@ -19,12 +19,12 @@
package org.apache.asterix.runtime.aggregates.collections;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -42,8 +42,7 @@
}
@Override
- public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
-
+ public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IAggregateEvaluator() {
private boolean first = true;
@@ -56,18 +55,18 @@
private final byte[] systemNullBytes = new byte[] { ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG };
@Override
- public void init() throws AlgebricksException {
+ public void init() throws HyracksDataException {
first = true;
}
@Override
- public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
if (!first) {
return;
}
eval.evaluate(tuple, inputVal);
byte typeTagByte = inputVal.getByteArray()[inputVal.getStartOffset()];
- if(typeTagByte == ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG){
+ if (typeTagByte == ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG) {
// Ignores SYSTEM_NULLs generated by local-first-element.
return;
}
@@ -76,7 +75,7 @@
}
@Override
- public void finish(IPointable result) throws AlgebricksException {
+ public void finish(IPointable result) throws HyracksDataException {
if (first) {
result.set(isLocal ? systemNullBytes : nullBytes, 0, 1);
return;
@@ -85,7 +84,7 @@
}
@Override
- public void finishPartial(IPointable result) throws AlgebricksException {
+ public void finishPartial(IPointable result) throws HyracksDataException {
finish(result);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
index aeb10a7..21d07b0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
@@ -23,7 +23,7 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -50,8 +50,7 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new ListifyAggregateFunctionEvalFactory(args, oltype);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateFunctionEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateFunctionEvalFactory.java
index 6a12423..1f1d408 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateFunctionEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateFunctionEvalFactory.java
@@ -22,12 +22,12 @@
import org.apache.asterix.builders.OrderedListBuilder;
import org.apache.asterix.om.types.AOrderedListType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -45,8 +45,7 @@
}
@Override
- public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
-
+ public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IAggregateEvaluator() {
private IPointable inputVal = new VoidPointable();
@@ -55,33 +54,33 @@
private OrderedListBuilder builder = new OrderedListBuilder();
@Override
- public void init() throws AlgebricksException {
+ public void init() throws HyracksDataException {
builder.reset(orderedlistType);
}
@Override
- public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
try {
eval.evaluate(tuple, inputVal);
builder.addItem(inputVal);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
@Override
- public void finish(IPointable result) throws AlgebricksException {
+ public void finish(IPointable result) throws HyracksDataException {
resultStorage.reset();
try {
builder.write(resultStorage.getDataOutput(), true);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
@Override
- public void finishPartial(IPointable result) throws AlgebricksException {
+ public void finishPartial(IPointable result) throws HyracksDataException {
finish(result);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/LocalFirstElementAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/LocalFirstElementAggregateDescriptor.java
index c1465d5..3070f86 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/LocalFirstElementAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/LocalFirstElementAggregateDescriptor.java
@@ -21,7 +21,7 @@
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -37,8 +37,7 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new FirstElementEvalFactory(args, true);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
index eed79ac..e6c6398 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
@@ -32,6 +32,7 @@
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.evaluators.ColumnAccessEvalFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public abstract class AbstractScalarAggregateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
@@ -39,7 +40,6 @@
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
throws AlgebricksException {
-
// The aggregate function will get a SingleFieldFrameTupleReference that points to the result of the ScanCollection.
// The list-item will always reside in the first field (column) of the SingleFieldFrameTupleReference.
IScalarEvaluatorFactory[] aggFuncArgs = new IScalarEvaluatorFactory[1];
@@ -55,7 +55,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
// Use ScanCollection to iterate over list items.
ScanCollectionUnnestingFunctionFactory scanCollectionFactory = new ScanCollectionUnnestingFunctionFactory(
args[0]);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/GenericScalarAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/GenericScalarAggregateFunction.java
index 8e14473..9cec06d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/GenericScalarAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/GenericScalarAggregateFunction.java
@@ -19,12 +19,12 @@
package org.apache.asterix.runtime.aggregates.scalar;
import org.apache.asterix.runtime.aggregates.base.SingleFieldFrameTupleReference;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -42,13 +42,13 @@
private final SingleFieldFrameTupleReference itemTuple = new SingleFieldFrameTupleReference();
public GenericScalarAggregateFunction(IAggregateEvaluator aggFunc, IUnnestingEvaluatorFactory scanCollectionFactory,
- IHyracksTaskContext context) throws AlgebricksException {
+ IHyracksTaskContext context) throws HyracksDataException {
this.aggFunc = aggFunc;
this.scanCollection = scanCollectionFactory.createUnnestingEvaluator(context);
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
scanCollection.init(tuple);
aggFunc.init();
while (scanCollection.step(listItemOut)) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
index 41a2d3f..6d20cca 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
@@ -36,6 +36,7 @@
import org.apache.asterix.om.base.AMutableDouble;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -44,13 +45,14 @@
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.evaluators.common.AccessibleByteArrayEval;
import org.apache.asterix.runtime.evaluators.common.ClosedRecordConstructorEvalFactory.ClosedRecordConstructorEval;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
@@ -89,34 +91,34 @@
.getSerializerDeserializer(BuiltinType.ANULL);
public AbstractSerializableAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
eval = args[0].createScalarEvaluator(context);
}
@Override
- public void init(DataOutput state) throws AlgebricksException {
+ public void init(DataOutput state) throws HyracksDataException {
try {
state.writeDouble(0.0);
state.writeLong(0);
state.writeByte(ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
@Override
- public abstract void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws AlgebricksException;
+ public abstract void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException;
@Override
- public abstract void finish(byte[] state, int start, int len, DataOutput result) throws AlgebricksException;
+ public abstract void finish(byte[] state, int start, int len, DataOutput result) throws HyracksDataException;
@Override
- public abstract void finishPartial(byte[] state, int start, int len, DataOutput result) throws AlgebricksException;
+ public abstract void finishPartial(byte[] state, int start, int len, DataOutput result) throws HyracksDataException;
protected abstract void processNull(byte[] state, int start);
protected void processDataValues(IFrameTupleReference tuple, byte[] state, int start, int len)
- throws AlgebricksException {
+ throws HyracksDataException {
if (skipStep(state, start)) {
return;
}
@@ -134,8 +136,7 @@
} else if (aggType == ATypeTag.SYSTEM_NULL) {
aggType = typeTag;
} else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
- throw new AlgebricksException(
- "Unexpected type " + typeTag + " in aggregation input stream. Expected type " + aggType + ".");
+ throw new IncompatibleTypeException(AsterixBuiltinFunctions.AVG, bytes[offset], aggType.serialize());
} else if (ATypeHierarchy.canPromote(aggType, typeTag)) {
aggType = typeTag;
}
@@ -171,9 +172,8 @@
sum += val;
break;
}
- default: {
- throw new NotImplementedException("Cannot compute AVG for values of type " + typeTag);
- }
+ default:
+ throw new UnsupportedItemTypeException(AsterixBuiltinFunctions.AVG, bytes[offset]);
}
BufferSerDeUtil.writeDouble(sum, state, start + SUM_OFFSET);
BufferSerDeUtil.writeLong(count, state, start + COUNT_OFFSET);
@@ -181,7 +181,7 @@
}
protected void finishPartialResults(byte[] state, int start, int len, DataOutput result)
- throws AlgebricksException {
+ throws HyracksDataException {
double sum = BufferSerDeUtil.getDouble(state, start + SUM_OFFSET);
long count = BufferSerDeUtil.getLong(state, start + COUNT_OFFSET);
ATypeTag aggType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(state[start + AGG_TYPE_OFFSET]);
@@ -210,19 +210,17 @@
result.write(avgBytes.getByteArray(), avgBytes.getStartOffset(), avgBytes.getLength());
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
protected void processPartialResults(IFrameTupleReference tuple, byte[] state, int start, int len)
- throws AlgebricksException {
+ throws HyracksDataException {
if (skipStep(state, start)) {
return;
}
double sum = BufferSerDeUtil.getDouble(state, start + SUM_OFFSET);
long count = BufferSerDeUtil.getLong(state, start + COUNT_OFFSET);
- ATypeTag aggType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(state[start + AGG_TYPE_OFFSET]);
-
eval.evaluate(tuple, inputVal);
byte[] serBytes = inputVal.getByteArray();
int offset = inputVal.getStartOffset();
@@ -239,7 +237,7 @@
}
case RECORD: {
// Expected.
- aggType = ATypeTag.DOUBLE;
+ ATypeTag aggType = ATypeTag.DOUBLE;
int nullBitmapSize = 0;
int offset1 = ARecordSerializerDeserializer.getFieldOffsetById(serBytes, offset, SUM_FIELD_ID,
nullBitmapSize, false);
@@ -253,14 +251,12 @@
state[start + AGG_TYPE_OFFSET] = aggType.serialize();
break;
}
- default: {
- throw new AlgebricksException("Global-Avg is not defined for values of type "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serBytes[offset]));
- }
+ default:
+ throw new UnsupportedItemTypeException(AsterixBuiltinFunctions.AVG, serBytes[offset]);
}
}
- protected void finishFinalResults(byte[] state, int start, int len, DataOutput result) throws AlgebricksException {
+ protected void finishFinalResults(byte[] state, int start, int len, DataOutput result) throws HyracksDataException {
double sum = BufferSerDeUtil.getDouble(state, start + SUM_OFFSET);
long count = BufferSerDeUtil.getLong(state, start + COUNT_OFFSET);
ATypeTag aggType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(state[start + AGG_TYPE_OFFSET]);
@@ -272,7 +268,7 @@
doubleSerde.serialize(aDouble, result);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableCountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableCountAggregateFunction.java
index d11997c..cca0d43 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableCountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableCountAggregateFunction.java
@@ -28,7 +28,7 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
@@ -56,22 +56,22 @@
private IScalarEvaluator eval;
public AbstractSerializableCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
eval = args[0].createScalarEvaluator(context);
}
@Override
- public void init(DataOutput state) throws AlgebricksException {
+ public void init(DataOutput state) throws HyracksDataException {
try {
state.writeBoolean(false);
state.writeLong(0);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
@Override
- public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
boolean metNull = BufferSerDeUtil.getBoolean(state, start);
long cnt = BufferSerDeUtil.getLong(state, start + 1);
eval.evaluate(tuple, inputVal);
@@ -87,7 +87,7 @@
}
@Override
- public void finish(byte[] state, int start, int len, DataOutput out) throws AlgebricksException {
+ public void finish(byte[] state, int start, int len, DataOutput out) throws HyracksDataException {
boolean metNull = BufferSerDeUtil.getBoolean(state, start);
long cnt = BufferSerDeUtil.getLong(state, start + 1);
try {
@@ -98,12 +98,12 @@
int64Serde.serialize(result, out);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
@Override
- public void finishPartial(byte[] state, int start, int len, DataOutput out) throws AlgebricksException {
+ public void finishPartial(byte[] state, int start, int len, DataOutput out) throws HyracksDataException {
finish(state, start, len, out);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableSumAggregateFunction.java
index d742d83..d5670a9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableSumAggregateFunction.java
@@ -35,17 +35,19 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.AMutableInt8;
import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -66,22 +68,22 @@
public ISerializerDeserializer serde;
public AbstractSerializableSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
eval = args[0].createScalarEvaluator(context);
}
@Override
- public void init(DataOutput state) throws AlgebricksException {
+ public void init(DataOutput state) throws HyracksDataException {
try {
state.writeByte(ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
state.writeDouble(0.0);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
@Override
- public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
if (skipStep(state, start)) {
return;
}
@@ -98,8 +100,7 @@
} else if (aggType == ATypeTag.SYSTEM_NULL) {
aggType = typeTag;
} else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
- throw new AlgebricksException("Unexpected type " + typeTag
- + " in aggregation input stream. Expected type (or a promotable type to)" + aggType + ".");
+ throw new IncompatibleTypeException(AsterixBuiltinFunctions.SUM, bytes[offset], aggType.serialize());
}
if (ATypeHierarchy.canPromote(aggType, typeTag)) {
@@ -145,9 +146,8 @@
processSystemNull();
break;
}
- default: {
- throw new NotImplementedException("Cannot compute SUM for values of type " + typeTag + ".");
- }
+ default:
+ throw new UnsupportedItemTypeException(AsterixBuiltinFunctions.SUM, bytes[offset]);
}
state[start + AGG_TYPE_OFFSET] = aggType.serialize();
BufferSerDeUtil.writeDouble(sum, state, start + SUM_OFFSET);
@@ -155,7 +155,7 @@
@SuppressWarnings("unchecked")
@Override
- public void finish(byte[] state, int start, int len, DataOutput out) throws AlgebricksException {
+ public void finish(byte[] state, int start, int len, DataOutput out) throws HyracksDataException {
ATypeTag aggType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(state[start + AGG_TYPE_OFFSET]);
double sum = BufferSerDeUtil.getDouble(state, start + SUM_OFFSET);
try {
@@ -206,16 +206,15 @@
break;
}
default:
- throw new AlgebricksException(
- "SumAggregationFunction: incompatible type for the result (" + aggType + "). ");
+ throw new UnsupportedItemTypeException(AsterixBuiltinFunctions.SUM, aggType.serialize());
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
@Override
- public void finishPartial(byte[] state, int start, int len, DataOutput out) throws AlgebricksException {
+ public void finishPartial(byte[] state, int start, int len, DataOutput out) throws HyracksDataException {
finish(state, start, len, out);
}
@@ -225,7 +224,7 @@
protected abstract void processNull(byte[] state, int start);
- protected abstract void processSystemNull() throws AlgebricksException;
+ protected abstract void processSystemNull() throws HyracksDataException;
protected abstract void finishSystemNull(DataOutput out) throws IOException;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java
index b05ff19..d1030ca 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java
@@ -23,6 +23,7 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
@@ -52,7 +53,7 @@
@Override
public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SerializableAvgAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateFunction.java
index 346fa76..4c32fe0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateFunction.java
@@ -22,7 +22,7 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -30,22 +30,22 @@
public class SerializableAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
public SerializableAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
processDataValues(tuple, state, start, len);
}
@Override
- public void finish(byte[] state, int start, int len, DataOutput result) throws AlgebricksException {
+ public void finish(byte[] state, int start, int len, DataOutput result) throws HyracksDataException {
finishFinalResults(state, start, len, result);
}
@Override
- public void finishPartial(byte[] state, int start, int len, DataOutput result) throws AlgebricksException {
+ public void finishPartial(byte[] state, int start, int len, DataOutput result) throws HyracksDataException {
finish(state, start, len, result);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java
index d60fb7f..d499523 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
/**
* count(NULL) returns NULL.
@@ -49,13 +49,13 @@
@Override
public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
- final IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ final IScalarEvaluatorFactory[] args) {
return new ISerializedAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SerializableCountAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateFunction.java
index 1dd9f90..fb8d8d7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateFunction.java
@@ -18,7 +18,7 @@
*/
package org.apache.asterix.runtime.aggregates.serializable.std;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -27,7 +27,7 @@
*/
public class SerializableCountAggregateFunction extends AbstractSerializableCountAggregateFunction {
public SerializableCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java
index 6f11b66..8ae70bb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java
@@ -23,12 +23,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SerializableGlobalAvgAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
@@ -47,13 +47,13 @@
@Override
public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
- final IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ final IScalarEvaluatorFactory[] args) {
return new ISerializedAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SerializableGlobalAvgAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateFunction.java
index 3895911..a916355 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateFunction.java
@@ -23,7 +23,7 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -31,22 +31,22 @@
public class SerializableGlobalAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
public SerializableGlobalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
processPartialResults(tuple, state, start, len);
}
@Override
- public void finish(byte[] state, int start, int len, DataOutput result) throws AlgebricksException {
+ public void finish(byte[] state, int start, int len, DataOutput result) throws HyracksDataException {
finishFinalResults(state, start, len, result);
}
@Override
- public void finishPartial(byte[] state, int start, int len, DataOutput result) throws AlgebricksException {
+ public void finishPartial(byte[] state, int start, int len, DataOutput result) throws HyracksDataException {
finishPartialResults(state, start, len, result);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateDescriptor.java
index b7507b1..a6c128e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateDescriptor.java
@@ -23,7 +23,7 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
@@ -48,13 +48,13 @@
@Override
public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
- final IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ final IScalarEvaluatorFactory[] args) {
return new ISerializedAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SerializableGlobalSqlAvgAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateFunction.java
index ebd8e89..8ddccdc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateFunction.java
@@ -21,7 +21,7 @@
import java.io.DataOutput;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -29,22 +29,22 @@
public class SerializableGlobalSqlAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
public SerializableGlobalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
processPartialResults(tuple, state, start, len);
}
@Override
- public void finish(byte[] state, int start, int len, DataOutput result) throws AlgebricksException {
+ public void finish(byte[] state, int start, int len, DataOutput result) throws HyracksDataException {
finishFinalResults(state, start, len, result);
}
@Override
- public void finishPartial(byte[] state, int start, int len, DataOutput result) throws AlgebricksException {
+ public void finishPartial(byte[] state, int start, int len, DataOutput result) throws HyracksDataException {
finishPartialResults(state, start, len, result);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateDescriptor.java
index e9920e2..1214b7c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateDescriptor.java
@@ -23,12 +23,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SerializableIntermediateAvgAggregateDescriptor
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
@@ -48,13 +48,13 @@
@Override
public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
- final IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ final IScalarEvaluatorFactory[] args) {
return new ISerializedAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SerializableIntermediateAvgAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateFunction.java
index af2f663..3d56133 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateFunction.java
@@ -23,7 +23,7 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -31,22 +31,22 @@
public class SerializableIntermediateAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
public SerializableIntermediateAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
processPartialResults(tuple, state, start, len);
}
@Override
- public void finish(byte[] state, int start, int len, DataOutput result) throws AlgebricksException {
+ public void finish(byte[] state, int start, int len, DataOutput result) throws HyracksDataException {
finishPartialResults(state, start, len, result);
}
@Override
- public void finishPartial(byte[] state, int start, int len, DataOutput result) throws AlgebricksException {
+ public void finishPartial(byte[] state, int start, int len, DataOutput result) throws HyracksDataException {
finishPartialResults(state, start, len, result);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateDescriptor.java
index aa4d699..6bfbdaa 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateDescriptor.java
@@ -23,12 +23,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SerializableIntermediateSqlAvgAggregateDescriptor
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
@@ -48,13 +48,13 @@
@Override
public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
- final IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ final IScalarEvaluatorFactory[] args) {
return new ISerializedAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SerializableIntermediateSqlAvgAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateFunction.java
index bd58214..f8449a5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateFunction.java
@@ -21,7 +21,7 @@
import java.io.DataOutput;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -29,22 +29,22 @@
public class SerializableIntermediateSqlAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
public SerializableIntermediateSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
processPartialResults(tuple, state, start, len);
}
@Override
- public void finish(byte[] state, int start, int len, DataOutput result) throws AlgebricksException {
+ public void finish(byte[] state, int start, int len, DataOutput result) throws HyracksDataException {
finishPartialResults(state, start, len, result);
}
@Override
- public void finishPartial(byte[] state, int start, int len, DataOutput result) throws AlgebricksException {
+ public void finishPartial(byte[] state, int start, int len, DataOutput result) throws HyracksDataException {
finishPartialResults(state, start, len, result);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java
index 8fd080a..2184ea3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java
@@ -23,12 +23,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SerializableLocalAvgAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
@@ -47,13 +47,13 @@
@Override
public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
- final IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ final IScalarEvaluatorFactory[] args) {
return new ISerializedAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SerializableLocalAvgAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateFunction.java
index aa9b385..0e7ba07 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateFunction.java
@@ -23,7 +23,7 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -31,22 +31,22 @@
public class SerializableLocalAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
public SerializableLocalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
processDataValues(tuple, state, start, len);
}
@Override
- public void finish(byte[] state, int start, int len, DataOutput result) throws AlgebricksException {
+ public void finish(byte[] state, int start, int len, DataOutput result) throws HyracksDataException {
finishPartialResults(state, start, len, result);
}
@Override
- public void finishPartial(byte[] state, int start, int len, DataOutput result) throws AlgebricksException {
+ public void finishPartial(byte[] state, int start, int len, DataOutput result) throws HyracksDataException {
finish(state, start, len, result);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateDescriptor.java
index f96a053..2c213cf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateDescriptor.java
@@ -23,12 +23,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SerializableLocalSqlAvgAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
@@ -47,13 +47,13 @@
@Override
public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
- final IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ final IScalarEvaluatorFactory[] args) {
return new ISerializedAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SerializableLocalSqlAvgAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateFunction.java
index 3609050..8925c31 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateFunction.java
@@ -21,7 +21,7 @@
import java.io.DataOutput;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -29,22 +29,22 @@
public class SerializableLocalSqlAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
public SerializableLocalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
processDataValues(tuple, state, start, len);
}
@Override
- public void finish(byte[] state, int start, int len, DataOutput result) throws AlgebricksException {
+ public void finish(byte[] state, int start, int len, DataOutput result) throws HyracksDataException {
finishPartialResults(state, start, len, result);
}
@Override
- public void finishPartial(byte[] state, int start, int len, DataOutput result) throws AlgebricksException {
+ public void finishPartial(byte[] state, int start, int len, DataOutput result) throws HyracksDataException {
finish(state, start, len, result);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateDescriptor.java
index 4392e6e..8ff8b53 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SerializableLocalSqlSumAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
@@ -46,13 +46,13 @@
@Override
public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
- final IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ final IScalarEvaluatorFactory[] args) {
return new ISerializedAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SerializableSqlSumAggregateFunction(args, true, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java
index e610077..b99c8c4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java
@@ -22,7 +22,7 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
@@ -46,13 +46,13 @@
@Override
public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
- final IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ final IScalarEvaluatorFactory[] args) {
return new ISerializedAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SerializableSumAggregateFunction(args, true, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateDescriptor.java
index ef1914a..f324141 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateDescriptor.java
@@ -22,7 +22,7 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
@@ -46,13 +46,13 @@
@Override
public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
- final IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ final IScalarEvaluatorFactory[] args) {
return new ISerializedAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SerializableSqlAvgAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateFunction.java
index 8b62efb..3ef7aeb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateFunction.java
@@ -20,7 +20,7 @@
import java.io.DataOutput;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -28,22 +28,22 @@
public class SerializableSqlAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
public SerializableSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws HyracksDataException {
processDataValues(tuple, state, start, len);
}
@Override
- public void finish(byte[] state, int start, int len, DataOutput result) throws AlgebricksException {
+ public void finish(byte[] state, int start, int len, DataOutput result) throws HyracksDataException {
finishFinalResults(state, start, len, result);
}
@Override
- public void finishPartial(byte[] state, int start, int len, DataOutput result) throws AlgebricksException {
+ public void finishPartial(byte[] state, int start, int len, DataOutput result) throws HyracksDataException {
finish(state, start, len, result);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateDescriptor.java
index 2a5bc53..43738d3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateDescriptor.java
@@ -22,7 +22,7 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
@@ -49,13 +49,13 @@
@Override
public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
- final IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ final IScalarEvaluatorFactory[] args) {
return new ISerializedAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SerializableSqlCountAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateFunction.java
index ce39c5a..257735f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateFunction.java
@@ -18,7 +18,7 @@
*/
package org.apache.asterix.runtime.aggregates.serializable.std;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -27,7 +27,7 @@
*/
public class SerializableSqlCountAggregateFunction extends AbstractSerializableCountAggregateFunction {
public SerializableSqlCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateDescriptor.java
index dec2688..9f62b28 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SerializableSqlSumAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
@@ -46,13 +46,13 @@
@Override
public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
- final IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ final IScalarEvaluatorFactory[] args) {
return new ISerializedAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SerializableSqlSumAggregateFunction(args, false, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateFunction.java
index f34d1be..2bd1f93 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateFunction.java
@@ -23,17 +23,19 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SerializableSqlSumAggregateFunction extends AbstractSerializableSumAggregateFunction {
private final boolean isLocalAgg;
public SerializableSqlSumAggregateFunction(IScalarEvaluatorFactory[] args, boolean isLocalAgg,
- IHyracksTaskContext context) throws AlgebricksException {
+ IHyracksTaskContext context) throws HyracksDataException {
super(args, context);
this.isLocalAgg = isLocalAgg;
}
@@ -43,12 +45,13 @@
}
@Override
- protected void processSystemNull() throws AlgebricksException {
+ protected void processSystemNull() throws HyracksDataException {
// For global aggregates simply ignore system null here,
// but if all input value are system null, then we should return
// null in finish().
if (isLocalAgg) {
- throw new AlgebricksException("Type SYSTEM_NULL encountered in local aggregate.");
+ throw new UnsupportedItemTypeException(AsterixBuiltinFunctions.SQL_SUM,
+ ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java
index 8a7cdef..d37d066 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SerializableSumAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
@@ -46,13 +46,13 @@
@Override
public ISerializedAggregateEvaluatorFactory createSerializableAggregateEvaluatorFactory(
- final IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ final IScalarEvaluatorFactory[] args) {
return new ISerializedAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SerializableSumAggregateFunction(args, false, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateFunction.java
index db3de24..e076183 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateFunction.java
@@ -23,18 +23,20 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SerializableSumAggregateFunction extends AbstractSerializableSumAggregateFunction {
private final boolean isLocalAgg;
public SerializableSumAggregateFunction(IScalarEvaluatorFactory[] args, boolean isLocalAgg,
- IHyracksTaskContext context) throws AlgebricksException {
+ IHyracksTaskContext context) throws HyracksDataException {
super(args, context);
this.isLocalAgg = isLocalAgg;
}
@@ -51,12 +53,13 @@
}
@Override
- protected void processSystemNull() throws AlgebricksException {
+ protected void processSystemNull() throws HyracksDataException {
// For global aggregates simply ignore system null here,
// but if all input value are system null, then we should return
// null in finish().
if (isLocalAgg) {
- throw new AlgebricksException("Type SYSTEM_NULL encountered in local aggregate.");
+ throw new UnsupportedItemTypeException(AsterixBuiltinFunctions.SUM,
+ ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
index 66e1af7..a59d461 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
@@ -36,6 +36,7 @@
import org.apache.asterix.om.base.AMutableDouble;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -44,13 +45,14 @@
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.evaluators.common.AccessibleByteArrayEval;
import org.apache.asterix.runtime.evaluators.common.ClosedRecordConstructorEvalFactory.ClosedRecordConstructorEval;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -92,33 +94,32 @@
.getSerializerDeserializer(BuiltinType.ANULL);
public AbstractAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
eval = args[0].createScalarEvaluator(context);
-
recType = new ARecordType(null, new String[] { "sum", "count" },
new IAType[] { BuiltinType.ADOUBLE, BuiltinType.AINT64 }, false);
recordEval = new ClosedRecordConstructorEval(recType, new IScalarEvaluator[] { evalSum, evalCount });
}
@Override
- public void init() {
+ public void init() throws HyracksDataException {
aggType = ATypeTag.SYSTEM_NULL;
sum = 0.0;
count = 0;
}
@Override
- public abstract void step(IFrameTupleReference tuple) throws AlgebricksException;
+ public abstract void step(IFrameTupleReference tuple) throws HyracksDataException;
@Override
- public abstract void finish(IPointable result) throws AlgebricksException;
+ public abstract void finish(IPointable result) throws HyracksDataException;
@Override
- public abstract void finishPartial(IPointable result) throws AlgebricksException;
+ public abstract void finishPartial(IPointable result) throws HyracksDataException;
protected abstract void processNull();
- protected void processDataValues(IFrameTupleReference tuple) throws AlgebricksException {
+ protected void processDataValues(IFrameTupleReference tuple) throws HyracksDataException {
if (skipStep()) {
return;
}
@@ -133,8 +134,7 @@
} else if (aggType == ATypeTag.SYSTEM_NULL) {
aggType = typeTag;
} else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
- throw new AlgebricksException(
- "Unexpected type " + typeTag + " in aggregation input stream. Expected type " + aggType + ".");
+ throw new IncompatibleTypeException(AsterixBuiltinFunctions.AVG, data[offset], aggType.serialize());
} else if (ATypeHierarchy.canPromote(aggType, typeTag)) {
aggType = typeTag;
}
@@ -171,12 +171,12 @@
break;
}
default: {
- throw new NotImplementedException("Cannot compute AVG for values of type " + typeTag);
+ throw new UnsupportedItemTypeException(AsterixBuiltinFunctions.AVG, data[offset]);
}
}
}
- protected void finishPartialResults(IPointable result) throws AlgebricksException {
+ protected void finishPartialResults(IPointable result) throws HyracksDataException {
resultStorage.reset();
try {
// Double check that count 0 is accounted
@@ -200,11 +200,11 @@
result.set(avgBytes);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
- protected void processPartialResults(IFrameTupleReference tuple) throws AlgebricksException {
+ protected void processPartialResults(IFrameTupleReference tuple) throws HyracksDataException {
if (skipStep()) {
return;
}
@@ -234,13 +234,12 @@
break;
}
default: {
- throw new AlgebricksException("Intermediate/global-Avg is not defined for values of type "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serBytes[offset]));
+ throw new UnsupportedItemTypeException("intermediate/global-Avg", serBytes[offset]);
}
}
}
- protected void finishFinalResults(IPointable result) throws AlgebricksException {
+ protected void finishFinalResults(IPointable result) throws HyracksDataException {
resultStorage.reset();
try {
if (count == 0 || aggType == ATypeTag.NULL) {
@@ -250,7 +249,7 @@
doubleSerde.serialize(aDouble, resultStorage.getDataOutput());
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractCountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractCountAggregateFunction.java
index ca44fa3..5814c73 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractCountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractCountAggregateFunction.java
@@ -26,7 +26,7 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -52,17 +52,17 @@
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
public AbstractCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
eval = args[0].createScalarEvaluator(context);
}
@Override
- public void init() {
+ public void init() throws HyracksDataException {
cnt = 0;
}
@Override
- public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
eval.evaluate(tuple, inputVal);
ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
.deserialize(inputVal.getByteArray()[inputVal.getStartOffset()]);
@@ -75,19 +75,19 @@
}
@Override
- public void finish(IPointable resultPointable) throws AlgebricksException {
+ public void finish(IPointable resultPointable) throws HyracksDataException {
resultStorage.reset();
try {
result.setValue(cnt);
int64Serde.serialize(result, resultStorage.getDataOutput());
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
resultPointable.set(resultStorage);
}
@Override
- public void finishPartial(IPointable resultPointable) throws AlgebricksException {
+ public void finishPartial(IPointable resultPointable) throws HyracksDataException {
finish(resultPointable);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractMinMaxAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractMinMaxAggregateFunction.java
index 5372cfb..1bc8efc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractMinMaxAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractMinMaxAggregateFunction.java
@@ -21,11 +21,11 @@
import java.io.IOException;
import org.apache.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
+import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.om.types.hierachy.ITypeConvertComputer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -51,19 +51,19 @@
private final boolean isMin;
public AbstractMinMaxAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isMin)
- throws AlgebricksException {
+ throws HyracksDataException {
eval = args[0].createScalarEvaluator(context);
this.isMin = isMin;
}
@Override
- public void init() {
+ public void init() throws HyracksDataException {
aggType = ATypeTag.SYSTEM_NULL;
tempValForCasting.reset();
}
@Override
- public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
if (skipStep()) {
return;
}
@@ -87,8 +87,7 @@
// Initialize min value.
outputVal.assign(inputVal);
} else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
- throw new AlgebricksException(
- "Unexpected type " + typeTag + " in aggregation input stream. Expected type " + aggType + ".");
+ throw new IncompatibleTypeException("min/max", typeTag.serialize(), aggType.serialize());
} else {
// If a system_null is encountered locally, it would be an error; otherwise if it is seen
@@ -109,17 +108,13 @@
tpc.convertType(outputVal.getByteArray(), outputVal.getStartOffset() + 1,
outputVal.getLength() - 1, tempValForCasting.getDataOutput());
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
outputVal.assign(tempValForCasting);
}
- try {
- if (cmp.compare(inputVal.getByteArray(), inputVal.getStartOffset(), inputVal.getLength(),
- outputVal.getByteArray(), outputVal.getStartOffset(), outputVal.getLength()) < 0) {
- outputVal.assign(inputVal);
- }
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
+ if (cmp.compare(inputVal.getByteArray(), inputVal.getStartOffset(), inputVal.getLength(),
+ outputVal.getByteArray(), outputVal.getStartOffset(), outputVal.getLength()) < 0) {
+ outputVal.assign(inputVal);
}
} else {
@@ -130,25 +125,17 @@
tpc.convertType(inputVal.getByteArray(), inputVal.getStartOffset() + 1,
inputVal.getLength() - 1, tempValForCasting.getDataOutput());
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
- try {
- if (cmp.compare(tempValForCasting.getByteArray(), tempValForCasting.getStartOffset(),
- tempValForCasting.getLength(), outputVal.getByteArray(), outputVal.getStartOffset(),
- outputVal.getLength()) < 0) {
- outputVal.assign(tempValForCasting);
- }
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
+ if (cmp.compare(tempValForCasting.getByteArray(), tempValForCasting.getStartOffset(),
+ tempValForCasting.getLength(), outputVal.getByteArray(), outputVal.getStartOffset(),
+ outputVal.getLength()) < 0) {
+ outputVal.assign(tempValForCasting);
}
} else {
- try {
- if (cmp.compare(inputVal.getByteArray(), inputVal.getStartOffset(), inputVal.getLength(),
- outputVal.getByteArray(), outputVal.getStartOffset(), outputVal.getLength()) < 0) {
- outputVal.assign(inputVal);
- }
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
+ if (cmp.compare(inputVal.getByteArray(), inputVal.getStartOffset(), inputVal.getLength(),
+ outputVal.getByteArray(), outputVal.getStartOffset(), outputVal.getLength()) < 0) {
+ outputVal.assign(inputVal);
}
}
@@ -157,7 +144,7 @@
}
@Override
- public void finish(IPointable result) throws AlgebricksException {
+ public void finish(IPointable result) throws HyracksDataException {
resultStorage.reset();
try {
switch (aggType) {
@@ -177,12 +164,12 @@
}
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
@Override
- public void finishPartial(IPointable result) throws AlgebricksException {
+ public void finishPartial(IPointable result) throws HyracksDataException {
finish(result);
}
@@ -192,7 +179,7 @@
protected abstract void processNull();
- protected abstract void processSystemNull() throws AlgebricksException;
+ protected abstract void processSystemNull() throws HyracksDataException;
protected abstract void finishSystemNull() throws IOException;
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractSumAggregateFunction.java
index 6938255..87a11a9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractSumAggregateFunction.java
@@ -34,17 +34,19 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.AMutableInt8;
import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -66,18 +68,18 @@
protected ISerializerDeserializer serde;
public AbstractSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
eval = args[0].createScalarEvaluator(context);
}
@Override
- public void init() {
+ public void init() throws HyracksDataException {
aggType = ATypeTag.SYSTEM_NULL;
sum = 0.0;
}
@Override
- public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
if (skipStep()) {
return;
}
@@ -92,8 +94,7 @@
} else if (aggType == ATypeTag.SYSTEM_NULL) {
aggType = typeTag;
} else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
- throw new AlgebricksException("Unexpected type " + typeTag
- + " in aggregation input stream. Expected type (or a promotable type to)" + aggType + ".");
+ throw new IncompatibleTypeException(AsterixBuiltinFunctions.SUM, typeTag.serialize(), aggType.serialize());
}
if (ATypeHierarchy.canPromote(aggType, typeTag)) {
@@ -136,14 +137,14 @@
break;
}
default: {
- throw new NotImplementedException("Cannot compute SUM for values of type " + typeTag + ".");
+ throw new UnsupportedItemTypeException(AsterixBuiltinFunctions.SUM, aggType.serialize());
}
}
}
@SuppressWarnings("unchecked")
@Override
- public void finish(IPointable result) throws AlgebricksException {
+ public void finish(IPointable result) throws HyracksDataException {
resultStorage.reset();
try {
switch (aggType) {
@@ -193,17 +194,16 @@
break;
}
default:
- throw new AlgebricksException(
- "SumAggregationFunction: incompatible type for the result (" + aggType + "). ");
+ throw new UnsupportedItemTypeException(AsterixBuiltinFunctions.SUM, aggType.serialize());
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
@Override
- public void finishPartial(IPointable result) throws AlgebricksException {
+ public void finishPartial(IPointable result) throws HyracksDataException {
finish(result);
}
@@ -213,7 +213,7 @@
protected abstract void processNull();
- protected abstract void processSystemNull() throws AlgebricksException;
+ protected abstract void processSystemNull() throws HyracksDataException;
protected abstract void finishSystemNull() throws IOException;
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java
index 2b5c9bb..e4204f1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java
@@ -23,12 +23,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class AvgAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
@@ -46,14 +46,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new AvgAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateFunction.java
index b5fd5ed..6408e56 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateFunction.java
@@ -20,7 +20,7 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.data.std.api.IPointable;
@@ -29,22 +29,22 @@
public class AvgAggregateFunction extends AbstractAvgAggregateFunction {
public AvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
processDataValues(tuple);
}
@Override
- public void finish(IPointable result) throws AlgebricksException {
+ public void finish(IPointable result) throws HyracksDataException {
finishFinalResults(result);
}
@Override
- public void finishPartial(IPointable result) throws AlgebricksException {
+ public void finishPartial(IPointable result) throws HyracksDataException {
finish(result);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateDescriptor.java
index f61da19..db9d1cb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
/**
* NULLs are also counted.
@@ -48,14 +48,12 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new CountAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateFunction.java
index 95fbca3..94b653d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateFunction.java
@@ -18,7 +18,7 @@
*/
package org.apache.asterix.runtime.aggregates.std;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -28,7 +28,7 @@
public class CountAggregateFunction extends AbstractCountAggregateFunction {
public CountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java
index edb55dc..0131eab 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java
@@ -23,12 +23,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class GlobalAvgAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
@@ -47,14 +47,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new GlobalAvgAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateFunction.java
index 3555863..30a2746 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateFunction.java
@@ -20,7 +20,7 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.data.std.api.IPointable;
@@ -29,22 +29,22 @@
public class GlobalAvgAggregateFunction extends AbstractAvgAggregateFunction {
public GlobalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
processPartialResults(tuple);
}
@Override
- public void finish(IPointable result) throws AlgebricksException {
+ public void finish(IPointable result) throws HyracksDataException {
finishFinalResults(result);
}
@Override
- public void finishPartial(IPointable result) throws AlgebricksException {
+ public void finishPartial(IPointable result) throws HyracksDataException {
finishPartialResults(result);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateDescriptor.java
index 4fdec36..a72c300 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateDescriptor.java
@@ -23,7 +23,7 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
@@ -47,14 +47,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new GlobalSqlAvgAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateFunction.java
index aa3d23d..893613b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateFunction.java
@@ -19,7 +19,7 @@
package org.apache.asterix.runtime.aggregates.std;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.data.std.api.IPointable;
@@ -28,22 +28,22 @@
public class GlobalSqlAvgAggregateFunction extends AbstractAvgAggregateFunction {
public GlobalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
processPartialResults(tuple);
}
@Override
- public void finish(IPointable result) throws AlgebricksException {
+ public void finish(IPointable result) throws HyracksDataException {
finishFinalResults(result);
}
@Override
- public void finishPartial(IPointable result) throws AlgebricksException {
+ public void finishPartial(IPointable result) throws HyracksDataException {
finishPartialResults(result);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateDescriptor.java
index 31fc059..3730405 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateDescriptor.java
@@ -23,7 +23,7 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
@@ -47,14 +47,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new IntermediateAvgAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateFunction.java
index 667ceda..ad64ef3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateFunction.java
@@ -20,7 +20,7 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.data.std.api.IPointable;
@@ -29,22 +29,22 @@
public class IntermediateAvgAggregateFunction extends AbstractAvgAggregateFunction {
public IntermediateAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
processPartialResults(tuple);
}
@Override
- public void finish(IPointable result) throws AlgebricksException {
+ public void finish(IPointable result) throws HyracksDataException {
finishPartialResults(result);
}
@Override
- public void finishPartial(IPointable result) throws AlgebricksException {
+ public void finishPartial(IPointable result) throws HyracksDataException {
finishPartialResults(result);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateDescriptor.java
index b58c14f..f5f40c2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateDescriptor.java
@@ -23,12 +23,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class IntermediateSqlAvgAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
@@ -46,14 +46,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new IntermediateSqlAvgAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateFunction.java
index 39da960..bc260e6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateFunction.java
@@ -19,7 +19,7 @@
package org.apache.asterix.runtime.aggregates.std;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.data.std.api.IPointable;
@@ -28,22 +28,22 @@
public class IntermediateSqlAvgAggregateFunction extends AbstractAvgAggregateFunction {
public IntermediateSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
processPartialResults(tuple);
}
@Override
- public void finish(IPointable result) throws AlgebricksException {
+ public void finish(IPointable result) throws HyracksDataException {
finishPartialResults(result);
}
@Override
- public void finishPartial(IPointable result) throws AlgebricksException {
+ public void finishPartial(IPointable result) throws HyracksDataException {
finishPartialResults(result);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java
index acb6493..acf415a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java
@@ -23,12 +23,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class LocalAvgAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
@@ -46,14 +46,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new LocalAvgAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateFunction.java
index 018a2ca..685af83 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateFunction.java
@@ -20,7 +20,7 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.data.std.api.IPointable;
@@ -29,22 +29,22 @@
public class LocalAvgAggregateFunction extends AbstractAvgAggregateFunction {
public LocalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
processDataValues(tuple);
}
@Override
- public void finish(IPointable result) throws AlgebricksException {
+ public void finish(IPointable result) throws HyracksDataException {
finishPartialResults(result);
}
@Override
- public void finishPartial(IPointable result) throws AlgebricksException {
+ public void finishPartial(IPointable result) throws HyracksDataException {
finish(result);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMaxAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMaxAggregateDescriptor.java
index 4b74121..c0336d9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMaxAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMaxAggregateDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class LocalMaxAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
@@ -47,14 +47,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new MinMaxAggregateFunction(args, ctx, false, true);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMinAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMinAggregateDescriptor.java
index 9c2a3e8..60f4d0d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMinAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMinAggregateDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class LocalMinAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
@@ -46,14 +46,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new MinMaxAggregateFunction(args, ctx, true, true);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateDescriptor.java
index 5e13b55..ed91001 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateDescriptor.java
@@ -23,7 +23,7 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
@@ -46,14 +46,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new LocalSqlAvgAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateFunction.java
index 1aaba7b..8c80c42 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateFunction.java
@@ -19,7 +19,7 @@
package org.apache.asterix.runtime.aggregates.std;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.data.std.api.IPointable;
@@ -28,22 +28,22 @@
public class LocalSqlAvgAggregateFunction extends AbstractAvgAggregateFunction {
public LocalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
processDataValues(tuple);
}
@Override
- public void finish(IPointable result) throws AlgebricksException {
+ public void finish(IPointable result) throws HyracksDataException {
finishPartialResults(result);
}
@Override
- public void finishPartial(IPointable result) throws AlgebricksException {
+ public void finishPartial(IPointable result) throws HyracksDataException {
finish(result);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMaxAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMaxAggregateDescriptor.java
index 4e5ba88..4768c22 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMaxAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMaxAggregateDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class LocalSqlMaxAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
@@ -47,14 +47,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SqlMinMaxAggregateFunction(args, ctx, false, true);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMinAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMinAggregateDescriptor.java
index 8a2036b..8ac1bb0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMinAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMinAggregateDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class LocalSqlMinAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
@@ -46,14 +46,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SqlMinMaxAggregateFunction(args, ctx, true, true);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateDescriptor.java
index b163428..28cea9f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class LocalSqlSumAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
@@ -46,14 +46,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SqlSumAggregateFunction(args, ctx, true);
};
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java
index 54c27da..3615ce8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class LocalSumAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
@@ -46,14 +46,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SumAggregateFunction(args, ctx, true);
};
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MaxAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MaxAggregateDescriptor.java
index bf136ea..daecb55 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MaxAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MaxAggregateDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class MaxAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
@@ -45,14 +45,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new MinMaxAggregateFunction(args, ctx, false, false);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinAggregateDescriptor.java
index f1fd7ce..d141959 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinAggregateDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class MinAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
@@ -45,14 +45,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new MinMaxAggregateFunction(args, ctx, true, false);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinMaxAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinMaxAggregateFunction.java
index 894f332..5d28018 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinMaxAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinMaxAggregateFunction.java
@@ -21,15 +21,16 @@
import java.io.IOException;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class MinMaxAggregateFunction extends AbstractMinMaxAggregateFunction {
private final boolean isLocalAgg;
public MinMaxAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isMin,
- boolean isLocalAgg) throws AlgebricksException {
+ boolean isLocalAgg) throws HyracksDataException {
super(args, context, isMin);
this.isLocalAgg = isLocalAgg;
}
@@ -45,9 +46,9 @@
}
@Override
- protected void processSystemNull() throws AlgebricksException {
+ protected void processSystemNull() throws HyracksDataException {
if (isLocalAgg) {
- throw new AlgebricksException("Type SYSTEM_NULL encountered in local aggregate.");
+ throw new UnsupportedItemTypeException("min/max", ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateDescriptor.java
index b83d638..9645120 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateDescriptor.java
@@ -23,12 +23,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SqlAvgAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
@@ -46,14 +46,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SqlAvgAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateFunction.java
index fd25d4d..e6c1233 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateFunction.java
@@ -19,7 +19,7 @@
package org.apache.asterix.runtime.aggregates.std;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.data.std.api.IPointable;
@@ -28,22 +28,22 @@
public class SqlAvgAggregateFunction extends AbstractAvgAggregateFunction {
public SqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
processDataValues(tuple);
}
@Override
- public void finish(IPointable result) throws AlgebricksException {
+ public void finish(IPointable result) throws HyracksDataException {
finishFinalResults(result);
}
@Override
- public void finishPartial(IPointable result) throws AlgebricksException {
+ public void finishPartial(IPointable result) throws HyracksDataException {
finishPartialResults(result);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateDescriptor.java
index 2b76854..317d713 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
/**
* NULLs are also counted.
@@ -48,14 +48,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new SqlCountAggregateFunction(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateFunction.java
index 45e9125..7396d39 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateFunction.java
@@ -18,7 +18,7 @@
*/
package org.apache.asterix.runtime.aggregates.std;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -28,7 +28,7 @@
public class SqlCountAggregateFunction extends AbstractCountAggregateFunction {
public SqlCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMaxAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMaxAggregateDescriptor.java
index cc553e2..22918e1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMaxAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMaxAggregateDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SqlMaxAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
@@ -45,14 +45,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SqlMinMaxAggregateFunction(args, ctx, false, false);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinAggregateDescriptor.java
index 9d34474..8c0e825 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinAggregateDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SqlMinAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
@@ -45,14 +45,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SqlMinMaxAggregateFunction(args, ctx, true, false);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinMaxAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinMaxAggregateFunction.java
index dd4ec85..2ad0e55 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinMaxAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinMaxAggregateFunction.java
@@ -21,7 +21,8 @@
import java.io.IOException;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -29,7 +30,7 @@
private final boolean isLocalAgg;
public SqlMinMaxAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isMin,
- boolean isLocalAgg) throws AlgebricksException {
+ boolean isLocalAgg) throws HyracksDataException {
super(args, context, isMin);
this.isLocalAgg = isLocalAgg;
}
@@ -39,9 +40,9 @@
}
@Override
- protected void processSystemNull() throws AlgebricksException {
+ protected void processSystemNull() throws HyracksDataException {
if (isLocalAgg) {
- throw new AlgebricksException("Type SYSTEM_NULL encountered in local aggregate.");
+ throw new UnsupportedItemTypeException("min/max", ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateDescriptor.java
index 9eee26a..a728261 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SqlSumAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
@@ -45,14 +45,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SqlSumAggregateFunction(args, ctx, false);
};
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateFunction.java
index 3536370..880c67e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateFunction.java
@@ -22,17 +22,19 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SqlSumAggregateFunction extends AbstractSumAggregateFunction {
private final boolean isLocalAgg;
public SqlSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isLocalAgg)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
this.isLocalAgg = isLocalAgg;
}
@@ -42,12 +44,13 @@
}
@Override
- protected void processSystemNull() throws AlgebricksException {
+ protected void processSystemNull() throws HyracksDataException {
// For global aggregates simply ignore system null here,
// but if all input value are system null, then we should return
// null in finish().
if (isLocalAgg) {
- throw new AlgebricksException("Type SYSTEM_NULL encountered in local aggregate.");
+ throw new UnsupportedItemTypeException(AsterixBuiltinFunctions.SQL_SUM,
+ ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateDescriptor.java
index be1b7a4..469d724 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SumAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
@@ -45,14 +45,13 @@
}
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new SumAggregateFunction(args, ctx, false);
};
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateFunction.java
index 87f3cb3..5efe79f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateFunction.java
@@ -22,17 +22,19 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SumAggregateFunction extends AbstractSumAggregateFunction {
private final boolean isLocalAgg;
public SumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isLocalAgg)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
this.isLocalAgg = isLocalAgg;
}
@@ -48,12 +50,13 @@
}
@Override
- protected void processSystemNull() throws AlgebricksException {
+ protected void processSystemNull() throws HyracksDataException {
// For global aggregates simply ignore system null here,
// but if all input value are system null, then we should return
// null in finish().
if (isLocalAgg) {
- throw new AlgebricksException("Type SYSTEM_NULL encountered in local aggregate.");
+ throw new UnsupportedItemTypeException(AsterixBuiltinFunctions.SUM,
+ ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java
index 998404f..5554e51 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java
@@ -25,7 +25,6 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
@@ -50,15 +49,14 @@
};
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new IAggregateEvaluator() {
@@ -70,30 +68,26 @@
boolean res = true;
@Override
- public void init() throws AlgebricksException {
+ public void init() throws HyracksDataException {
res = true;
}
@Override
- public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
res = false;
}
@SuppressWarnings("unchecked")
@Override
- public void finish(IPointable result) throws AlgebricksException {
+ public void finish(IPointable result) throws HyracksDataException {
resultStorage.reset();
ABoolean b = res ? ABoolean.TRUE : ABoolean.FALSE;
- try {
- serde.serialize(b, resultStorage.getDataOutput());
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ serde.serialize(b, resultStorage.getDataOutput());
result.set(resultStorage);
}
@Override
- public void finishPartial(IPointable result) throws AlgebricksException {
+ public void finishPartial(IPointable result) throws HyracksDataException {
finish(result);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java
index 3d484b2..5f38351 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java
@@ -25,7 +25,6 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
@@ -49,15 +48,13 @@
};
@Override
- public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) {
return new IAggregateEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new IAggregateEvaluator() {
@@ -69,30 +66,26 @@
boolean res = false;
@Override
- public void init() throws AlgebricksException {
+ public void init() throws HyracksDataException {
res = false;
}
@Override
- public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
res = true;
}
@SuppressWarnings("unchecked")
@Override
- public void finish(IPointable result) throws AlgebricksException {
+ public void finish(IPointable result) throws HyracksDataException {
resultStorage.reset();
ABoolean b = res ? ABoolean.TRUE : ABoolean.FALSE;
- try {
- serde.serialize(b, resultStorage.getDataOutput());
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ serde.serialize(b, resultStorage.getDataOutput());
result.set(resultStorage);
}
@Override
- public void finishPartial(IPointable result) throws AlgebricksException {
+ public void finishPartial(IPointable result) throws HyracksDataException {
finish(result);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/AsterixTupleFilter.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/AsterixTupleFilter.java
index 341261d..2712a47 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/AsterixTupleFilter.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/AsterixTupleFilter.java
@@ -19,11 +19,11 @@
package org.apache.asterix.runtime.base;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.data.IBinaryBooleanInspector;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -35,9 +35,9 @@
private final IPointable p = VoidPointable.FACTORY.createPointable();
public AsterixTupleFilter(IHyracksTaskContext ctx, IScalarEvaluatorFactory evalFactory,
- IBinaryBooleanInspector boolInspector) throws AlgebricksException {
+ IBinaryBooleanInspector boolInspector) throws HyracksDataException {
this.boolInspector = boolInspector;
- eval = evalFactory.createScalarEvaluator(ctx);
+ this.eval = evalFactory.createScalarEvaluator(ctx);
}
@Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java
index 7e9b74c..37d99c3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java
@@ -27,18 +27,19 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.AMutablePoint;
import org.apache.asterix.om.base.APoint;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -57,14 +58,13 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -76,7 +76,7 @@
.getSerializerDeserializer(BuiltinType.APOINT);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -93,11 +93,11 @@
aPoint.setValue(cX, cY);
pointSerde.serialize(aPoint, out);
} else {
- throw new AlgebricksException("get-center does not support the type: "
- + bytes[startOffset] + " It is only implemented for CIRCLE.");
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java
index 9ed2802..121a426 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java
@@ -26,18 +26,19 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.ADouble;
import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -55,14 +56,13 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -74,7 +74,7 @@
.getSerializerDeserializer(BuiltinType.ADOUBLE);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -88,11 +88,11 @@
aDouble.setValue(radius);
doubleSerde.serialize(aDouble, out);
} else {
- throw new AlgebricksException("get-radius does not support the type: "
- + bytes[startOffset] + " It is only implemented for CIRCLE.");
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java
index e5f74c4..eeaebb2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java
@@ -38,7 +38,8 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -62,14 +63,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -85,7 +84,7 @@
.getSerializerDeserializer(BuiltinType.APOINT);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -138,7 +137,8 @@
startOffset + APolygonSerializerDeserializer.getNumberOfPointsOffset());
if (numOfPoints < 3) {
- throw new HyracksDataException("Polygon must have at least 3 points.");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
}
listBuilder.reset(pointListType);
for (int i = 0; i < numOfPoints; ++i) {
@@ -153,12 +153,12 @@
}
listBuilder.write(out, true);
} else {
- throw new AlgebricksException(
- "get-points does not support the type: " + bytes[startOffset]
- + " It is only implemented for LINE, RECTANGLE, or POLYGON.");
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_LINE_TYPE_TAG, ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG,
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java
index 878747b..c69e21c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java
@@ -27,18 +27,19 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.ADouble;
import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -56,14 +57,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -75,7 +74,7 @@
.getSerializerDeserializer(BuiltinType.ADOUBLE);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -89,11 +88,11 @@
aDouble.setValue(x);
doubleSerde.serialize(aDouble, out);
} else {
- throw new AlgebricksException("get-x does not support the type: " + bytes[startOffset]
- + " It is only implemented for POINT.");
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java
index 99e396c..dc57ff7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java
@@ -27,18 +27,19 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.ADouble;
import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -59,14 +60,13 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -78,7 +78,7 @@
.getSerializerDeserializer(BuiltinType.ADOUBLE);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -92,11 +92,11 @@
aDouble.setValue(y);
doubleSerde.serialize(aDouble, out);
} else {
- throw new AlgebricksException("get-y does not support the type: " + bytes[startOffset]
- + " It is only implemented for POINT.");
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalDayAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalDayAccessor.java
index 3732954..6ab2aa3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalDayAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalDayAccessor.java
@@ -29,18 +29,19 @@
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -62,14 +63,12 @@
* @see org.apache.asterix.om.function.IFunctionDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory[])
*/
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -85,7 +84,7 @@
private final AMutableInt64 aMutableInt64 = new AMutableInt64(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -114,7 +113,10 @@
} else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
} else {
- throw new AlgebricksException("Inapplicable input type: " + bytes[startOffset]);
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
int year = calSystem.getYear(chrononTimeInMs);
@@ -124,7 +126,7 @@
aMutableInt64.setValue(day);
intSerde.serialize(aMutableInt64, out);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalHourAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalHourAccessor.java
index dc10d71..e915ad6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalHourAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalHourAccessor.java
@@ -29,18 +29,19 @@
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -63,14 +64,13 @@
* @see org.apache.asterix.om.function.IFunctionDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory[])
*/
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -86,7 +86,7 @@
private final AMutableInt64 aMutableInt64 = new AMutableInt64(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -114,14 +114,17 @@
} else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
} else {
- throw new AlgebricksException("Inapplicable input type: " + bytes[startOffset]);
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_TIME_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
int hour = calSystem.getHourOfDay(chrononTimeInMs);
aMutableInt64.setValue(hour);
intSerde.serialize(aMutableInt64, out);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndAccessor.java
index 54ad490..a8771ad 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndAccessor.java
@@ -34,14 +34,15 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -60,14 +61,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -89,7 +88,7 @@
private final AMutableTime aTime = new AMutableTime(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -109,13 +108,16 @@
} else if (timeType == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
aDateTime.setValue(endTime);
datetimeSerde.serialize(aDateTime, out);
+ } else {
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
} else {
- throw new AlgebricksException(FID.getName() + ": expects NULL/INTERVAL, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]));
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDateAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDateAccessor.java
index ae7b7f1..09f7526 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDateAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDateAccessor.java
@@ -30,14 +30,15 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -55,14 +56,13 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -76,7 +76,7 @@
private final AMutableDate aDate = new AMutableDate(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -91,17 +91,15 @@
aDate.setValue((int) (endTime));
dateSerde.serialize(aDate, out);
} else {
- throw new AlgebricksException(FID.getName()
- + ": expects NULL/INTERVAL(of DATE), but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]));
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
} else {
- throw new AlgebricksException(FID.getName()
- + ": expects NULL/INTERVAL(of DATE), but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]));
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDatetimeAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDatetimeAccessor.java
index 0c28a6d..f1b2511 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDatetimeAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDatetimeAccessor.java
@@ -30,14 +30,15 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -57,14 +58,13 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -78,7 +78,7 @@
private final AMutableDateTime aDateTime = new AMutableDateTime(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -93,17 +93,15 @@
aDateTime.setValue(endTime);
datetimeSerde.serialize(aDateTime, out);
} else {
- throw new AlgebricksException(FID.getName()
- + ": expects NULL/INTERVAL(of DATETIME), but got INTERVAL(of "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(timeType) + ")");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
} else {
- throw new AlgebricksException(FID.getName()
- + ": expects NULL/INTERVAL(of DATETIME), but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]));
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndTimeAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndTimeAccessor.java
index 1008d9b..d0af90b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndTimeAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndTimeAccessor.java
@@ -30,14 +30,15 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -55,14 +56,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -76,7 +75,7 @@
private final AMutableTime aTime = new AMutableTime(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -91,17 +90,15 @@
aTime.setValue((int) (endTime));
timeSerde.serialize(aTime, out);
} else {
- throw new AlgebricksException(FID.getName()
- + ": expects NULL/INTERVAL(of TIME), but got INTERVAL(of "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(timeType) + ")");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
} else {
- throw new AlgebricksException(FID.getName()
- + ": expects NULL/INTERVAL(of TIME), but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]));
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartAccessor.java
index 7ab7f27..b6e1eac 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartAccessor.java
@@ -34,14 +34,15 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -60,13 +61,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -88,7 +88,7 @@
private final AMutableTime aTime = new AMutableTime(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -109,13 +109,16 @@
} else if (timeType == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
aDateTime.setValue(startTime);
datetimeSerde.serialize(aDateTime, out);
+ } else {
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
} else {
- throw new AlgebricksException(FID.getName() + ": expects NULL/INTERVAL, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]));
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDateAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDateAccessor.java
index 35b17a2..3e5a82f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDateAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDateAccessor.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.runtime.evaluators.accessors;
+import static org.apache.asterix.om.types.ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG;
+
import java.io.DataOutput;
import java.io.IOException;
@@ -30,14 +32,15 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -56,13 +59,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -76,14 +78,14 @@
private final AMutableDate aDate = new AMutableDate(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
resultStorage.reset();
try {
- if (bytes[startOffset] == ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG) {
+ if (bytes[startOffset] == SERIALIZED_INTERVAL_TYPE_TAG) {
byte timeType = AIntervalSerializerDeserializer.getIntervalTimeType(bytes,
startOffset + 1);
long startTime = AIntervalSerializerDeserializer.getIntervalStart(bytes,
@@ -92,17 +94,15 @@
aDate.setValue((int) (startTime));
dateSerde.serialize(aDate, out);
} else {
- throw new AlgebricksException(FID.getName()
- + ": expects NULL/INTERVAL(of DATE), but got INTERVAL(of "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(timeType) + ")");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
} else {
- throw new AlgebricksException(FID.getName()
- + ": expects NULL/INTERVAL(of DATE), but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]));
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDatetimeAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDatetimeAccessor.java
index 7a1376b..7beff69 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDatetimeAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDatetimeAccessor.java
@@ -30,14 +30,15 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -55,14 +56,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -76,7 +75,7 @@
private final AMutableDateTime aDateTime = new AMutableDateTime(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -92,17 +91,15 @@
aDateTime.setValue(startTime);
datetimeSerde.serialize(aDateTime, out);
} else {
- throw new AlgebricksException(FID.getName()
- + ": expects NULL/INTERVAL(of DATETIME), but got INTERVAL(of "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(timeType) + ")");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
} else {
- throw new AlgebricksException(FID.getName()
- + ": expects NULL/INTERVAL(of DATETIME), but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]));
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartTimeAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartTimeAccessor.java
index ae4fb4d..0c35e1f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartTimeAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartTimeAccessor.java
@@ -30,14 +30,15 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -56,13 +57,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -76,7 +76,7 @@
private final AMutableTime aTime = new AMutableTime(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -92,17 +92,15 @@
aTime.setValue((int) (startTime));
timeSerde.serialize(aTime, out);
} else {
- throw new AlgebricksException(FID.getName()
- + ": expects NULL/INTERVAL(of TIME), but got INTERVAL(of "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset])
- + ")");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
} else {
- throw new AlgebricksException(FID.getName() + ": expects NULL/INTERVAL, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]));
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMillisecondAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMillisecondAccessor.java
index 453aae0..f86fe94 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMillisecondAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMillisecondAccessor.java
@@ -29,18 +29,19 @@
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -62,14 +63,12 @@
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
*/
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -85,7 +84,7 @@
private final AMutableInt64 aMutableInt64 = new AMutableInt64(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -113,14 +112,17 @@
} else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
} else {
- throw new AlgebricksException("Inapplicable input type: " + bytes[startOffset]);
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_TIME_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
int ms = calSystem.getMillisOfSec(chrononTimeInMs);
aMutableInt64.setValue(ms);
intSerde.serialize(aMutableInt64, out);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMinuteAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMinuteAccessor.java
index 6455c4f..00fde98 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMinuteAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMinuteAccessor.java
@@ -29,18 +29,19 @@
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -65,14 +66,12 @@
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
*/
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -88,7 +87,7 @@
private final AMutableInt64 aMutableInt64 = new AMutableInt64(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -116,14 +115,17 @@
} else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
} else {
- throw new AlgebricksException("Inapplicable input type: " + bytes[startOffset]);
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_TIME_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
int min = calSystem.getMinOfHour(chrononTimeInMs);
aMutableInt64.setValue(min);
intSerde.serialize(aMutableInt64, out);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMonthAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMonthAccessor.java
index e6fb01f..bbb3cd6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMonthAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMonthAccessor.java
@@ -29,18 +29,19 @@
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -62,14 +63,12 @@
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
*/
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -85,7 +84,7 @@
private final AMutableInt64 aMutableInt64 = new AMutableInt64(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -114,7 +113,10 @@
} else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
} else {
- throw new AlgebricksException("Inapplicable input type: " + bytes[startOffset]);
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
int year = calSystem.getYear(chrononTimeInMs);
@@ -123,7 +125,7 @@
aMutableInt64.setValue(month);
intSerde.serialize(aMutableInt64, out);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalSecondAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalSecondAccessor.java
index e14d08d..815db31 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalSecondAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalSecondAccessor.java
@@ -29,18 +29,19 @@
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -62,14 +63,12 @@
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
*/
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -85,7 +84,7 @@
private final AMutableInt64 aMutableInt64 = new AMutableInt64(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -114,15 +113,16 @@
} else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
} else {
- throw new AlgebricksException("Inapplicable input type: " + bytes[startOffset]);
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_TIME_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
-
int sec = calSystem.getSecOfMin(chrononTimeInMs);
-
aMutableInt64.setValue(sec);
intSerde.serialize(aMutableInt64, out);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalYearAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalYearAccessor.java
index 41c123b..187770c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalYearAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalYearAccessor.java
@@ -29,18 +29,19 @@
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -60,14 +61,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -86,7 +85,7 @@
private final AMutableInt64 aMutableInt64 = new AMutableInt64(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int startOffset = argPtr.getStartOffset();
@@ -133,16 +132,19 @@
result.set(resultStorage);
return;
} else {
- throw new AlgebricksException("Inapplicable input type: " + bytes[startOffset]);
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG,
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
int year = calSystem.getYear(chrononTimeInMs);
-
aMutableInt64.setValue(year);
intSerde.serialize(aMutableInt64, out);
result.set(resultStorage);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/AbstractTypeCheckEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/AbstractTypeCheckEvaluator.java
index 2866f7b..0fa6c0e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/AbstractTypeCheckEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/AbstractTypeCheckEvaluator.java
@@ -24,7 +24,6 @@
import org.apache.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
@@ -52,7 +51,7 @@
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
Value match = isMatch(argPtr.getByteArray()[argPtr.getStartOffset()]);
if (match == Value.MISSING) {
@@ -60,13 +59,9 @@
return;
}
ABoolean res = match == Value.TRUE ? ABoolean.TRUE : ABoolean.FALSE;
- try {
- resultStorage.reset();
- aObjSerDer.serialize(res, out);
- result.set(resultStorage);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ resultStorage.reset();
+ aObjSerDer.serialize(res, out);
+ result.set(resultStorage);
}
protected abstract Value isMatch(byte typeTag);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/AccessibleByteArrayEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/AccessibleByteArrayEval.java
index f621241..bc7cc03 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/AccessibleByteArrayEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/AccessibleByteArrayEval.java
@@ -21,8 +21,8 @@
import java.io.DataOutput;
import java.io.DataOutputStream;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -42,14 +42,10 @@
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
result.set(baaos.getByteArray(), 0, baaos.size());
}
- public ByteArrayAccessibleOutputStream getBaaos() {
- return baaos;
- }
-
public void reset() {
baaos.reset();
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/AsterixListAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/AsterixListAccessor.java
index c16d3d1..aa5b0b5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/AsterixListAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/AsterixListAccessor.java
@@ -28,6 +28,8 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
/**
* Utility class for accessing serialized unordered and ordered lists.
@@ -52,15 +54,18 @@
return itemType == ATypeTag.ANY;
}
- public void reset(byte[] listBytes, int start) throws AsterixException {
+ public void reset(byte[] listBytes, int start) throws HyracksDataException {
this.listBytes = listBytes;
this.start = start;
- listType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(listBytes[start]);
- if (listType != ATypeTag.UNORDEREDLIST && listType != ATypeTag.ORDEREDLIST) {
- throw new AsterixException("Unsupported type: " + listType);
+ byte typeTag = listBytes[start];
+ if (typeTag != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG
+ && typeTag != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
+ throw new TypeMismatchException("list-accessor", 0, listBytes[start],
+ ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG, ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
}
+ listType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(listBytes[start]);
itemType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(listBytes[start + 1]);
- if (listType == ATypeTag.UNORDEREDLIST) {
+ if (listBytes[start] == ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
size = AUnorderedListSerializerDeserializer.getNumberOfItems(listBytes, start);
} else {
size = AOrderedListSerializerDeserializer.getNumberOfItems(listBytes, start);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ClosedRecordConstructorEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ClosedRecordConstructorEvalFactory.java
index f0ce758..1dd7089 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ClosedRecordConstructorEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ClosedRecordConstructorEvalFactory.java
@@ -24,7 +24,6 @@
import org.apache.asterix.builders.IARecordBuilder;
import org.apache.asterix.builders.RecordBuilder;
import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -47,7 +46,7 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
int n = args.length / 2;
IScalarEvaluator[] evalFields = new IScalarEvaluator[n];
for (int i = 0; i < n; i++) {
@@ -71,23 +70,19 @@
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
- try {
- resultStorage.reset();
- if (first) {
- first = false;
- recBuilder.reset(this.recType);
- }
- recBuilder.init();
- for (int i = 0; i < evalFields.length; i++) {
- evalFields[i].evaluate(tuple, fieldValuePointable);
- recBuilder.addField(i, fieldValuePointable);
- }
- recBuilder.write(out, true);
- result.set(resultStorage);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ if (first) {
+ first = false;
+ recBuilder.reset(this.recType);
}
+ recBuilder.init();
+ for (int i = 0; i < evalFields.length; i++) {
+ evalFields[i].evaluate(tuple, fieldValuePointable);
+ recBuilder.addField(i, fieldValuePointable);
+ }
+ recBuilder.write(out, true);
+ result.set(resultStorage);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/CreateMBREvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/CreateMBREvalFactory.java
index 0a4d258..a1eb8ad 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/CreateMBREvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/CreateMBREvalFactory.java
@@ -30,9 +30,10 @@
import org.apache.asterix.dataflow.data.nontagged.serde.APointSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -59,7 +60,7 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
@@ -72,7 +73,7 @@
private IScalarEvaluator eval2 = coordinateEvalFactory.createScalarEvaluator(ctx);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval0.evaluate(tuple, inputArg0);
eval1.evaluate(tuple, inputArg1);
eval2.evaluate(tuple, inputArg2);
@@ -98,17 +99,18 @@
result.set(resultStorage);
return;
}
-
resultStorage.reset();
- // type-check: (Point/Line/Polygon/Circle/Rectangle/Null, Int32, Int32)
- if (data1[startOffset1] != ATypeTag.SERIALIZED_INT32_TYPE_TAG
- || data2[startOffset2] != ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
- throw new AlgebricksException(
- "Expects Types: (Point/Line/Polygon/Circle/Rectangle/Null, Int32, Int32).");
+ if (data1[startOffset1] != ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
+ throw new TypeMismatchException(AsterixBuiltinFunctions.CREATE_MBR, 1, data1[startOffset1],
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG);
}
-
int dimension = AInt32SerializerDeserializer.getInt(data1, startOffset1 + 1);
- int coordinate = AInt32SerializerDeserializer.getInt(inputArg2.getByteArray(), startOffset2 + 1);
+
+ if (data2[startOffset2] != ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
+ throw new TypeMismatchException(AsterixBuiltinFunctions.CREATE_MBR, 2, data2[startOffset2],
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG);
+ }
+ int coordinate = AInt32SerializerDeserializer.getInt(data2, startOffset2 + 1);
double value;
if (dimension == 2) {
ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data0[startOffset0]);
@@ -293,7 +295,6 @@
}
break;
case RECTANGLE:
- value = Double.MAX_VALUE;
switch (coordinate) {
case 0: {
value = ADoubleSerializerDeserializer.getDouble(data0,
@@ -326,20 +327,18 @@
}
break;
default:
- throw new NotImplementedException(
- "create-mbr is only implemented for POINT, LINE, POLYGON, CIRCLE and RECTANGLE. Encountered type: "
- + tag + ".");
-
+ throw new TypeMismatchException(AsterixBuiltinFunctions.CREATE_MBR, 0,
+ data0[startOffset0], ATypeTag.SERIALIZED_POINT_TYPE_TAG,
+ ATypeTag.SERIALIZED_LINE_TYPE_TAG, ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
+ ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG, ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
}
} else {
throw new NotImplementedException(dimension + "D is not supported");
}
out.writeByte(ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
out.writeDouble(value);
- } catch (HyracksDataException hde) {
- throw new AlgebricksException(hde);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceCheckEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceCheckEvaluator.java
index 3dca584..b9129a6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceCheckEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceCheckEvaluator.java
@@ -29,7 +29,7 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -52,7 +52,7 @@
.getSerializerDeserializer(BuiltinType.ABOOLEAN);
public EditDistanceCheckEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
edThreshEval = args[2].createScalarEvaluator(context);
listBuilder = new OrderedListBuilder();
@@ -60,9 +60,8 @@
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
-
firstStringEval.evaluate(tuple, argPtr1);
firstTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER
.deserialize(argPtr1.getByteArray()[argPtr1.getStartOffset()]);
@@ -76,18 +75,19 @@
return;
}
try {
- edThresh = ATypeHierarchy.getIntegerValue(argPtrThreshold.getByteArray(), argPtrThreshold.getStartOffset());
+ edThresh = ATypeHierarchy.getIntegerValue(AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK.getName(), 2,
+ argPtrThreshold.getByteArray(), argPtrThreshold.getStartOffset());
editDistance = computeResult(argPtr1, argPtr2, firstTypeTag);
writeResult(editDistance);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
@Override
protected int computeResult(IPointable left, IPointable right, ATypeTag argType)
- throws AlgebricksException, HyracksDataException {
+ throws HyracksDataException {
byte[] leftBytes = left.getByteArray();
int leftStartOffset = left.getStartOffset();
byte[] rightBytes = right.getByteArray();
@@ -105,8 +105,8 @@
}
default: {
- throw new AlgebricksException(AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK.getName()
- + ": expects input type as STRING or ORDEREDLIST but got " + argType + ".");
+ throw new TypeMismatchException(AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK, 0, argType.serialize(),
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG, ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceContainsEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceContainsEvaluator.java
index 23a01d8..e108916 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceContainsEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceContainsEvaluator.java
@@ -18,9 +18,9 @@
*/
package org.apache.asterix.runtime.evaluators.common;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +29,12 @@
public class EditDistanceContainsEvaluator extends EditDistanceCheckEvaluator {
public EditDistanceContainsEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- protected int computeResult(IPointable left, IPointable right, ATypeTag argType) throws AlgebricksException {
+ protected int computeResult(IPointable left, IPointable right, ATypeTag argType) throws HyracksDataException {
byte[] leftBytes = left.getByteArray();
int leftStartOffset = left.getStartOffset();
byte[] rightBytes = right.getByteArray();
@@ -46,17 +46,13 @@
rightStartOffset + typeIndicatorSize, edThresh);
}
case ORDEREDLIST: {
- try {
- firstOrdListIter.reset(leftBytes, leftStartOffset);
- secondOrdListIter.reset(rightBytes, rightStartOffset);
- return ed.getSimilarityContains(firstOrdListIter, secondOrdListIter, edThresh);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ firstOrdListIter.reset(leftBytes, leftStartOffset);
+ secondOrdListIter.reset(rightBytes, rightStartOffset);
+ return ed.getSimilarityContains(firstOrdListIter, secondOrdListIter, edThresh);
}
default: {
- throw new AlgebricksException(AsterixBuiltinFunctions.EDIT_DISTANCE_CONTAINS.getName()
- + ": expects input type as STRING or ORDEREDLIST but got " + argType + ".");
+ throw new TypeMismatchException(AsterixBuiltinFunctions.EDIT_DISTANCE_CONTAINS, 0, argType.serialize(),
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG, ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
}
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceEvaluator.java
index d2adc1c..efab277 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceEvaluator.java
@@ -25,10 +25,13 @@
import org.apache.asterix.fuzzyjoin.similarity.SimilarityMetricEditDistance;
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -64,15 +67,14 @@
protected ATypeTag secondTypeTag;
public EditDistanceEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
firstStringEval = args[0].createScalarEvaluator(context);
secondStringEval = args[1].createScalarEvaluator(context);
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
-
firstStringEval.evaluate(tuple, argPtr1);
firstTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER
.deserialize(argPtr1.getByteArray()[argPtr1.getStartOffset()]);
@@ -85,22 +87,17 @@
return;
}
- try {
- editDistance = computeResult(argPtr1, argPtr2, firstTypeTag);
- } catch (HyracksDataException e1) {
- throw new AlgebricksException(e1);
- }
-
+ editDistance = computeResult(argPtr1, argPtr2, firstTypeTag);
try {
writeResult(editDistance);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
protected int computeResult(IPointable left, IPointable right, ATypeTag argType)
- throws AlgebricksException, HyracksDataException {
+ throws HyracksDataException {
byte[] leftBytes = left.getByteArray();
int leftStartOffset = left.getStartOffset();
byte[] rightBytes = right.getByteArray();
@@ -114,43 +111,38 @@
case ORDEREDLIST: {
firstOrdListIter.reset(leftBytes, leftStartOffset);
secondOrdListIter.reset(rightBytes, rightStartOffset);
- try {
- return (int) ed.getSimilarity(firstOrdListIter, secondOrdListIter);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ return (int) ed.getSimilarity(firstOrdListIter, secondOrdListIter);
}
-
default: {
- throw new AlgebricksException(
- "Invalid type " + argType + " passed as argument to edit distance function.");
+ throw new TypeMismatchException(AsterixBuiltinFunctions.EDIT_DISTANCE, 0, argType.serialize(),
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG, ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
}
}
}
- protected boolean checkArgTypes(ATypeTag typeTag1, ATypeTag typeTag2) throws AlgebricksException {
+ protected boolean checkArgTypes(ATypeTag typeTag1, ATypeTag typeTag2) throws HyracksDataException {
if (typeTag1 != typeTag2) {
- throw new AlgebricksException(
- "Incompatible argument types given in edit distance: " + typeTag1 + " " + typeTag2);
+ throw new IncompatibleTypeException(AsterixBuiltinFunctions.EDIT_DISTANCE, typeTag1.serialize(),
+ typeTag2.serialize());
}
// Since they are equal, check one tag is enough.
if (typeTag1 != ATypeTag.STRING && typeTag1 != ATypeTag.ORDEREDLIST) { // could be an list
- throw new AlgebricksException(
- "Only String or OrderedList type are allowed in edit distance, but given : " + typeTag1);
+ throw new TypeMismatchException(AsterixBuiltinFunctions.EDIT_DISTANCE, 0, typeTag1.serialize(),
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG, ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
}
if (typeTag1 == ATypeTag.ORDEREDLIST) {
itemTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER
.deserialize(argPtr1.getByteArray()[argPtr1.getStartOffset() + 1]);
if (itemTypeTag == ATypeTag.ANY) {
- throw new AlgebricksException("\n Edit Distance can only be called on homogenous lists");
+ throw new UnsupportedItemTypeException(AsterixBuiltinFunctions.EDIT_DISTANCE, itemTypeTag.serialize());
}
itemTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER
.deserialize(argPtr2.getByteArray()[argPtr2.getStartOffset() + 1]);
if (itemTypeTag == ATypeTag.ANY) {
- throw new AlgebricksException("\n Edit Distance can only be called on homogenous lists");
+ throw new UnsupportedItemTypeException(AsterixBuiltinFunctions.EDIT_DISTANCE, itemTypeTag.serialize());
}
}
return true;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/GramTokensEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/GramTokensEvaluator.java
index ed20b26..f41f02e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/GramTokensEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/GramTokensEvaluator.java
@@ -22,10 +22,10 @@
import java.io.IOException;
import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -57,7 +57,7 @@
private final AOrderedListType listType;
public GramTokensEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, IBinaryTokenizer tokenizer,
- BuiltinType itemType) throws AlgebricksException {
+ BuiltinType itemType) throws HyracksDataException {
stringEval = args[0].createScalarEvaluator(context);
gramLengthEval = args[1].createScalarEvaluator(context);
prePostEval = args[2].createScalarEvaluator(context);
@@ -66,19 +66,14 @@
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
stringEval.evaluate(tuple, stringArg);
gramLengthEval.evaluate(tuple, gramLengthArg);
prePostEval.evaluate(tuple, prePostArg);
- int gramLength = 0;
- try {
- gramLength = ATypeHierarchy.getIntegerValue(gramLengthArg.getByteArray(), gramLengthArg.getStartOffset());
- } catch (HyracksDataException e1) {
- throw new AlgebricksException(e1);
- }
-
+ int gramLength = ATypeHierarchy.getIntegerValue(AsterixBuiltinFunctions.GRAM_TOKENS.getName(), 1,
+ gramLengthArg.getByteArray(), gramLengthArg.getStartOffset());
tokenizer.setGramlength(gramLength);
boolean prePost = BooleanPointable.getBoolean(prePostArg.getByteArray(),
prePostArg.getStartOffset() + typeIndicatorSize);
@@ -93,7 +88,7 @@
}
listBuilder.write(out, true);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityFiltersCache.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityFiltersCache.java
index 52988a0..37b06af 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityFiltersCache.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityFiltersCache.java
@@ -24,7 +24,6 @@
import org.apache.asterix.fuzzyjoin.similarity.SimilarityFilters;
import org.apache.asterix.fuzzyjoin.similarity.SimilarityFiltersFactory;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
@@ -40,16 +39,11 @@
private SimilarityFilters similarityFiltersCached = null;
public SimilarityFilters get(float similarityThreshold, byte[] similarityNameBytes, int startOffset, int len)
- throws AlgebricksException {
+ throws HyracksDataException {
if (similarityThreshold != similarityThresholdCached || similarityNameBytesCached == null
|| !Arrays.equals(similarityNameBytes, similarityNameBytesCached)) {
bbis.setByteBuffer(ByteBuffer.wrap(similarityNameBytes), startOffset + 1);
- String similarityName;
- try {
- similarityName = utf8SerDer.deserialize(dis);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ String similarityName = utf8SerDer.deserialize(dis);
similarityNameBytesCached = Arrays.copyOfRange(similarityNameBytes, startOffset, len);
similarityFiltersCached = SimilarityFiltersFactory.getSimilarityFilters(similarityName,
similarityThreshold);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardCheckEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardCheckEvaluator.java
index f31b289..bcbaf0c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardCheckEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardCheckEvaluator.java
@@ -28,7 +28,6 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.functions.BinaryHashMap.BinaryEntry;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -54,7 +53,7 @@
protected final AOrderedListType listType = new AOrderedListType(BuiltinType.ANY, "list");
public SimilarityJaccardCheckEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
jaccThreshEval = args[2].createScalarEvaluator(context);
listBuilder = new OrderedListBuilder();
@@ -62,7 +61,7 @@
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
firstOrdListEval.evaluate(tuple, argPtr1);
@@ -86,7 +85,7 @@
result.set(resultStorage);
return;
}
- if (prepareLists(argPtr1, argPtr2, firstTypeTag)) {
+ if (prepareLists(argPtr1, argPtr2)) {
jaccSim = computeResult();
} else {
jaccSim = 0.0f;
@@ -94,7 +93,7 @@
try {
writeResult(jaccSim);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardEvaluator.java
index 01c4b6b..01acf25 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardEvaluator.java
@@ -27,12 +27,13 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.AFloat;
import org.apache.asterix.om.base.AMutableFloat;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.functions.BinaryHashMap;
import org.apache.asterix.runtime.evaluators.functions.BinaryHashMap.BinaryEntry;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -89,7 +90,7 @@
protected final boolean ignoreCase = true;
public SimilarityJaccardEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
firstOrdListEval = args[0].createScalarEvaluator(context);
secondOrdListEval = args[1].createScalarEvaluator(context);
byte[] emptyValBuf = new byte[8];
@@ -98,7 +99,7 @@
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
firstOrdListEval.evaluate(tuple, argPtr1);
@@ -118,7 +119,7 @@
result.set(resultStorage);
return;
}
- if (prepareLists(argPtr1, argPtr2, firstTypeTag)) {
+ if (prepareLists(argPtr1, argPtr2)) {
jaccSim = computeResult();
} else {
jaccSim = 0.0f;
@@ -126,18 +127,14 @@
try {
writeResult(jaccSim);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
- protected boolean prepareLists(IPointable left, IPointable right, ATypeTag argType) throws AlgebricksException {
- try {
- firstListIter.reset(left.getByteArray(), left.getStartOffset());
- secondListIter.reset(right.getByteArray(), right.getStartOffset());
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ protected boolean prepareLists(IPointable left, IPointable right) throws HyracksDataException {
+ firstListIter.reset(left.getByteArray(), left.getStartOffset());
+ secondListIter.reset(right.getByteArray(), right.getStartOffset());
// Check for special case where one of the lists is empty, since list
// types won't match.
if (firstListIter.size() == 0 || secondListIter.size() == 0) {
@@ -147,7 +144,7 @@
return true;
}
- protected float computeResult() throws AlgebricksException {
+ protected float computeResult() throws HyracksDataException {
// We will subtract the intersection size later to get the real union size.
int firstListSize = firstListIter.size();
int secondListSize = secondListIter.size();
@@ -161,18 +158,14 @@
ATypeTag probeItemTypeTag = (probeList == firstListIter) ? firstItemTypeTag : secondItemTypeTag;
setHashMap(buildItemTypeTag, probeItemTypeTag);
- try {
- buildHashMap(buildList);
- int intersectionSize = probeHashMap(probeList, buildListSize, probeListSize);
- // Special indicator for the "check" version of jaccard.
- if (intersectionSize < 0) {
- return -1;
- }
- unionSize -= intersectionSize;
- return (float) intersectionSize / (float) unionSize;
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
+ buildHashMap(buildList);
+ int intersectionSize = probeHashMap(probeList, buildListSize, probeListSize);
+ // Special indicator for the "check" version of jaccard.
+ if (intersectionSize < 0) {
+ return -1;
}
+ unionSize -= intersectionSize;
+ return (float) intersectionSize / (float) unionSize;
}
protected void buildHashMap(AbstractAsterixListIterator buildIter) throws HyracksDataException {
@@ -239,7 +232,7 @@
hashMap = new BinaryHashMap(TABLE_SIZE, TABLE_FRAME_SIZE, putHashFunc, getHashFunc, cmp);
}
- protected boolean checkArgTypes(ATypeTag typeTag1, ATypeTag typeTag2) throws AlgebricksException {
+ protected boolean checkArgTypes(ATypeTag typeTag1, ATypeTag typeTag2) throws HyracksDataException {
switch (typeTag1) {
case ORDEREDLIST: {
firstListIter = fstOrdListIter;
@@ -250,7 +243,8 @@
break;
}
default: {
- throw new AlgebricksException("Invalid types " + typeTag1 + " given as arguments to jaccard.");
+ throw new TypeMismatchException(AsterixBuiltinFunctions.SIMILARITY_JACCARD, 0, typeTag1.serialize(),
+ ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG, ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
}
}
switch (typeTag2) {
@@ -263,7 +257,8 @@
break;
}
default: {
- throw new AlgebricksException("Invalid types " + typeTag2 + " given as arguments to jaccard.");
+ throw new TypeMismatchException(AsterixBuiltinFunctions.SIMILARITY_JACCARD, 1, typeTag2.serialize(),
+ ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG, ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
}
}
return true;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardPrefixEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardPrefixEvaluator.java
index 5ab9e35..d1060c6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardPrefixEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardPrefixEvaluator.java
@@ -32,11 +32,11 @@
import org.apache.asterix.fuzzyjoin.similarity.SimilarityMetric;
import org.apache.asterix.om.base.AFloat;
import org.apache.asterix.om.base.AMutableFloat;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -76,7 +76,7 @@
.getSerializerDeserializer(BuiltinType.AFLOAT);
public SimilarityJaccardPrefixEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
evalLen1 = args[0].createScalarEvaluator(context);
evalTokens1 = args[1].createScalarEvaluator(context);
evalLen2 = args[2].createScalarEvaluator(context);
@@ -86,7 +86,7 @@
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
// similarity threshold
sim = 0;
@@ -100,21 +100,11 @@
}
evalLen1.evaluate(tuple, inputVal);
- int length1 = 0;
- try {
- length1 = ATypeHierarchy.getIntegerValue(inputVal.getByteArray(), inputVal.getStartOffset());
- } catch (HyracksDataException e1) {
- throw new AlgebricksException(e1);
- }
-
+ int length1 = ATypeHierarchy.getIntegerValue(AsterixBuiltinFunctions.SIMILARITY_JACCARD.getName(), 0,
+ inputVal.getByteArray(), inputVal.getStartOffset());
evalLen2.evaluate(tuple, inputVal);
- int length2 = 0;
-
- try {
- length2 = ATypeHierarchy.getIntegerValue(inputVal.getByteArray(), inputVal.getStartOffset());
- } catch (HyracksDataException e1) {
- throw new AlgebricksException(e1);
- }
+ int length2 = ATypeHierarchy.getIntegerValue(AsterixBuiltinFunctions.SIMILARITY_JACCARD.getName(), 2,
+ inputVal.getByteArray(), inputVal.getStartOffset());
//
// -- - length filter - --
@@ -130,8 +120,8 @@
int startOffset = inputVal.getStartOffset();
if (serList[startOffset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
&& serList[startOffset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
- throw new AlgebricksException("Scan collection is not defined for values of type"
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serList[startOffset]));
+ throw new TypeMismatchException(AsterixBuiltinFunctions.SIMILARITY_JACCARD, 1, serList[startOffset],
+ ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG, ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
}
int lengthTokens1;
@@ -145,15 +135,11 @@
try {
itemOffset = AOrderedListSerializerDeserializer.getItemOffset(serList, startOffset, i);
} catch (AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
-
- try {
- token = ATypeHierarchy.getIntegerValueWithDifferentTypeTagPosition(serList, itemOffset,
+ token = ATypeHierarchy.getIntegerValueWithDifferentTypeTagPosition(
+ AsterixBuiltinFunctions.SIMILARITY_JACCARD.getName(), 1, serList, itemOffset,
startOffset + 1);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
tokens1.add(token);
}
} else {
@@ -167,15 +153,12 @@
try {
itemOffset = AUnorderedListSerializerDeserializer.getItemOffset(serList, startOffset, i);
} catch (AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
- try {
- token = ATypeHierarchy.getIntegerValueWithDifferentTypeTagPosition(serList, itemOffset,
+ token = ATypeHierarchy.getIntegerValueWithDifferentTypeTagPosition(
+ AsterixBuiltinFunctions.SIMILARITY_JACCARD.getName(), 1, serList, itemOffset,
startOffset + 1);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
tokens1.add(token);
}
}
@@ -192,8 +175,8 @@
startOffset = inputVal.getStartOffset();
if (serList[startOffset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
&& serList[startOffset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
- throw new AlgebricksException("Scan collection is not defined for values of type"
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serList[startOffset]));
+ throw new TypeMismatchException(AsterixBuiltinFunctions.SIMILARITY_JACCARD, 3, serList[startOffset],
+ ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG, ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
}
int lengthTokens2;
@@ -208,15 +191,11 @@
try {
itemOffset = AOrderedListSerializerDeserializer.getItemOffset(serList, startOffset, i);
} catch (AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
-
- try {
- token = ATypeHierarchy.getIntegerValueWithDifferentTypeTagPosition(serList, itemOffset,
+ token = ATypeHierarchy.getIntegerValueWithDifferentTypeTagPosition(
+ AsterixBuiltinFunctions.SIMILARITY_JACCARD.getName(), 3, serList, itemOffset,
startOffset + 1);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
tokens2.add(token);
}
} else {
@@ -230,15 +209,11 @@
try {
itemOffset = AUnorderedListSerializerDeserializer.getItemOffset(serList, startOffset, i);
} catch (AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
-
- try {
- token = ATypeHierarchy.getIntegerValueWithDifferentTypeTagPosition(serList, itemOffset,
+ token = ATypeHierarchy.getIntegerValueWithDifferentTypeTagPosition(
+ AsterixBuiltinFunctions.SIMILARITY_JACCARD.getName(), 3, serList, itemOffset,
startOffset + 1);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
tokens2.add(token);
}
}
@@ -249,13 +224,8 @@
// -- - token prefix - --
evalTokenPrefix.evaluate(tuple, inputVal);
- int tokenPrefix = 0;
-
- try {
- tokenPrefix = ATypeHierarchy.getIntegerValue(inputVal.getByteArray(), inputVal.getStartOffset());
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ int tokenPrefix = ATypeHierarchy.getIntegerValue(AsterixBuiltinFunctions.SIMILARITY_JACCARD.getName(), 4,
+ inputVal.getByteArray(), inputVal.getStartOffset());
//
// -- - position filter - --
@@ -281,12 +251,12 @@
try {
writeResult();
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
- public void writeResult() throws AlgebricksException, IOException {
+ public void writeResult() throws IOException {
res.setValue(sim);
reusSerde.serialize(res, out);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardSortedCheckEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardSortedCheckEvaluator.java
index d0b0f7d..19e9395 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardSortedCheckEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardSortedCheckEvaluator.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.evaluators.common;
import org.apache.asterix.fuzzyjoin.similarity.SimilarityMetricJaccard;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,16 +28,12 @@
protected final SimilarityMetricJaccard jaccard = new SimilarityMetricJaccard();
public SimilarityJaccardSortedCheckEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- protected float computeResult() throws AlgebricksException {
- try {
- return jaccard.getSimilarity(firstListIter, secondListIter, jaccThresh);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ protected float computeResult() throws HyracksDataException {
+ return jaccard.getSimilarity(firstListIter, secondListIter, jaccThresh);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardSortedEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardSortedEvaluator.java
index 242a59b..d40cb67 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardSortedEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SimilarityJaccardSortedEvaluator.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.evaluators.common;
import org.apache.asterix.fuzzyjoin.similarity.SimilarityMetricJaccard;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -30,16 +29,12 @@
protected final SimilarityMetricJaccard jaccard = new SimilarityMetricJaccard();
public SimilarityJaccardSortedEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
}
@Override
- protected float computeResult() throws AlgebricksException {
- try {
- return jaccard.getSimilarity(firstListIter, secondListIter);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ protected float computeResult() throws HyracksDataException {
+ return jaccard.getSimilarity(firstListIter, secondListIter);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/WordTokensEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/WordTokensEvaluator.java
index faace0e..e51d5cf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/WordTokensEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/WordTokensEvaluator.java
@@ -24,10 +24,10 @@
import org.apache.asterix.builders.OrderedListBuilder;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -45,14 +45,14 @@
private final AOrderedListType listType;
public WordTokensEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, IBinaryTokenizer tokenizer,
- BuiltinType itemType) throws AlgebricksException {
+ BuiltinType itemType) throws HyracksDataException {
stringEval = args[0].createScalarEvaluator(context);
this.tokenizer = tokenizer;
this.listType = new AOrderedListType(itemType, null);
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
stringEval.evaluate(tuple, argPtr);
tokenizer.reset(argPtr.getByteArray(), argPtr.getStartOffset(), argPtr.getLength());
@@ -64,7 +64,7 @@
}
listBuilder.write(out, true);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
index 0ed89dd..60fc3e0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
@@ -27,7 +27,7 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -59,13 +59,13 @@
.getSerializerDeserializer(BuiltinType.ANULL);
public AbstractComparisonEvaluator(IScalarEvaluatorFactory evalLeftFactory,
- IScalarEvaluatorFactory evalRightFactory, IHyracksTaskContext context) throws AlgebricksException {
+ IScalarEvaluatorFactory evalRightFactory, IHyracksTaskContext context) throws HyracksDataException {
this.evalLeft = evalLeftFactory.createScalarEvaluator(context);
this.evalRight = evalRightFactory.createScalarEvaluator(context);
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
// Evaluates input args.
@@ -80,19 +80,15 @@
checkTotallyOrderable();
}
- try {
- // Checks whether two types are comparable
- if (comparabilityCheck()) {
- // Two types can be compared
- int r = compareResults();
- ABoolean b = getComparisonResult(r) ? ABoolean.TRUE : ABoolean.FALSE;
- serde.serialize(b, out);
- } else {
- // result:NULL - two types cannot be compared.
- nullSerde.serialize(ANull.NULL, out);
- }
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
+ // Checks whether two types are comparable
+ if (comparabilityCheck()) {
+ // Two types can be compared
+ int r = compareResults();
+ ABoolean b = getComparisonResult(r) ? ABoolean.TRUE : ABoolean.FALSE;
+ serde.serialize(b, out);
+ } else {
+ // result:NULL - two types cannot be compared.
+ nullSerde.serialize(ANull.NULL, out);
}
result.set(resultStorage);
}
@@ -103,7 +99,7 @@
// checks whether we can apply >, >=, <, and <= operations to the given type since
// these operations can not be defined for certain types.
- protected void checkTotallyOrderable() throws AlgebricksException {
+ protected void checkTotallyOrderable() throws HyracksDataException {
if (argLeft.getLength() != 0) {
ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
switch (typeTag) {
@@ -115,8 +111,7 @@
case POLYGON:
case CIRCLE:
case RECTANGLE:
- throw new AlgebricksException(
- "Comparison operations (GT, GE, LT, and LE) for the " + typeTag + " type are not defined.");
+ throw new UnsupportedTypeException(ComparisonHelper.COMPARISON, argLeft.getTag());
default:
return;
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
index 8044c83..4597621 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
@@ -37,6 +37,8 @@
import org.apache.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
import org.apache.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -47,6 +49,7 @@
public class ComparisonHelper implements Serializable {
private static final long serialVersionUID = 1L;
+ static final String COMPARISON = "comparison operations (>, >=, <, and <=)";
private final IBinaryComparator strBinaryComp = AqlBinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE
.createBinaryComparator();
@@ -98,10 +101,9 @@
private int compareStrongTypedWithArg(ATypeTag expectedTypeTag, ATypeTag actualTypeTag, IPointable arg1,
IPointable arg2) throws HyracksDataException {
if (expectedTypeTag != actualTypeTag) {
- throw new HyracksDataException(
- "Comparison is undefined between " + expectedTypeTag + " and " + actualTypeTag + ".");
+ throw new IncompatibleTypeException(COMPARISON, actualTypeTag.serialize(), expectedTypeTag.serialize());
}
- int result = 0;
+ int result;
byte[] leftBytes = arg1.getByteArray();
int leftOffset = arg1.getStartOffset();
int leftLen = arg1.getLength() - 1;
@@ -109,64 +111,56 @@
int rightOffset = arg2.getStartOffset();
int rightLen = arg2.getLength() - 1;
- try {
- switch (actualTypeTag) {
- case YEARMONTHDURATION:
- case TIME:
- case DATE:
- result = Integer.compare(AInt32SerializerDeserializer.getInt(leftBytes, leftOffset),
- AInt32SerializerDeserializer.getInt(rightBytes, rightOffset));
- break;
- case DAYTIMEDURATION:
- case DATETIME:
- result = Long.compare(AInt64SerializerDeserializer.getLong(leftBytes, leftOffset),
- AInt64SerializerDeserializer.getLong(rightBytes, rightOffset));
- break;
- case CIRCLE:
- result = circleBinaryComp.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
- rightLen);
- break;
- case LINE:
- result = lineBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
- rightLen);
- break;
- case POINT:
- result = pointBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
- rightLen);
- break;
- case POINT3D:
- result = point3DBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
- rightLen);
- break;
- case POLYGON:
- result = polygonBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
- rightLen);
- break;
- case DURATION:
- result = durationBinaryComp.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
- rightLen);
- break;
- case INTERVAL:
- result = intervalBinaryComp.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
- rightLen);
- break;
- case RECTANGLE:
- result = rectangleBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
- rightLen);
- break;
- case BINARY:
- result = byteArrayComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
- rightLen);
- break;
- case UUID:
- result = uuidBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
- rightLen);
- break;
- default:
- throw new HyracksDataException("Comparison for " + actualTypeTag + " is not supported.");
- }
- } catch (HyracksDataException e) {
- throw new HyracksDataException(e);
+ switch (actualTypeTag) {
+ case YEARMONTHDURATION:
+ case TIME:
+ case DATE:
+ result = Integer.compare(AInt32SerializerDeserializer.getInt(leftBytes, leftOffset),
+ AInt32SerializerDeserializer.getInt(rightBytes, rightOffset));
+ break;
+ case DAYTIMEDURATION:
+ case DATETIME:
+ result = Long.compare(AInt64SerializerDeserializer.getLong(leftBytes, leftOffset),
+ AInt64SerializerDeserializer.getLong(rightBytes, rightOffset));
+ break;
+ case CIRCLE:
+ result = circleBinaryComp.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset, rightLen);
+ break;
+ case LINE:
+ result = lineBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+ rightLen);
+ break;
+ case POINT:
+ result = pointBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+ rightLen);
+ break;
+ case POINT3D:
+ result = point3DBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+ rightLen);
+ break;
+ case POLYGON:
+ result = polygonBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+ rightLen);
+ break;
+ case DURATION:
+ result = durationBinaryComp.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset, rightLen);
+ break;
+ case INTERVAL:
+ result = intervalBinaryComp.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset, rightLen);
+ break;
+ case RECTANGLE:
+ result = rectangleBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+ rightLen);
+ break;
+ case BINARY:
+ result = byteArrayComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset, rightLen);
+ break;
+ case UUID:
+ result = uuidBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+ rightLen);
+ break;
+ default:
+ throw new UnsupportedTypeException(COMPARISON, actualTypeTag.serialize());
}
return result;
}
@@ -177,21 +171,15 @@
byte b1 = arg2.getByteArray()[arg2.getStartOffset()];
return compareByte(b0, b1);
}
- throw new HyracksDataException("Comparison is undefined between types ABoolean and " + typeTag2 + " .");
+ throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG, typeTag2.serialize());
}
private int compareStringWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws HyracksDataException {
if (typeTag2 == ATypeTag.STRING) {
- int result;
- try {
- result = strBinaryComp.compare(arg1.getByteArray(), arg1.getStartOffset(), arg1.getLength() - 1,
+ return strBinaryComp.compare(arg1.getByteArray(), arg1.getStartOffset(), arg1.getLength() - 1,
arg2.getByteArray(), arg2.getStartOffset(), arg2.getLength() - 1);
- } catch (HyracksDataException e) {
- throw new HyracksDataException(e);
- }
- return result;
}
- throw new HyracksDataException("Comparison is undefined between types AString and " + typeTag2 + " .");
+ throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_STRING_TYPE_TAG, typeTag2.serialize());
}
private int compareDoubleWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws HyracksDataException {
@@ -202,32 +190,21 @@
double s = ADoubleSerializerDeserializer.getDouble(leftBytes, leftOffset);
switch (typeTag2) {
- case INT8: {
- byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightOffset);
- return compareDouble(s, v2);
- }
- case INT16: {
- short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightOffset);
- return compareDouble(s, v2);
- }
- case INT32: {
- int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightOffset);
- return compareDouble(s, v2);
- }
- case INT64: {
- long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightOffset);
- return compareDouble(s, v2);
- }
- case FLOAT: {
- float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset);
- return compareDouble(s, v2);
- }
- case DOUBLE: {
- double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset);
- return compareDouble(s, v2);
- }
+ case INT8:
+ return compareDouble(s, AInt8SerializerDeserializer.getByte(rightBytes, rightOffset));
+ case INT16:
+ return compareDouble(s, AInt16SerializerDeserializer.getShort(rightBytes, rightOffset));
+ case INT32:
+ return compareDouble(s, AInt32SerializerDeserializer.getInt(rightBytes, rightOffset));
+ case INT64:
+ return compareDouble(s, AInt64SerializerDeserializer.getLong(rightBytes, rightOffset));
+ case FLOAT:
+ return compareDouble(s, AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset));
+ case DOUBLE:
+ return compareDouble(s, ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset));
default: {
- throw new HyracksDataException("Comparison is undefined between types ADouble and " + typeTag2 + " .");
+ throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG,
+ typeTag2.serialize());
}
}
}
@@ -240,33 +217,21 @@
float s = FloatPointable.getFloat(leftBytes, leftOffset);
switch (typeTag2) {
- case INT8: {
- byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightOffset);
- return compareFloat(s, v2);
- }
- case INT16: {
- short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightOffset);
- return compareFloat(s, v2);
- }
- case INT32: {
- int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightOffset);
- return compareFloat(s, v2);
- }
- case INT64: {
- long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightOffset);
- return compareFloat(s, v2);
- }
- case FLOAT: {
- float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset);
- return compareFloat(s, v2);
- }
- case DOUBLE: {
- double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset);
- return compareDouble(s, v2);
- }
- default: {
- throw new HyracksDataException("Comparison is undefined between types AFloat and " + typeTag2 + " .");
- }
+ case INT8:
+ return compareFloat(s, AInt8SerializerDeserializer.getByte(rightBytes, rightOffset));
+ case INT16:
+ return compareFloat(s, AInt16SerializerDeserializer.getShort(rightBytes, rightOffset));
+ case INT32:
+ return compareFloat(s, AInt32SerializerDeserializer.getInt(rightBytes, rightOffset));
+ case INT64:
+ return compareFloat(s, AInt64SerializerDeserializer.getLong(rightBytes, rightOffset));
+ case FLOAT:
+ return compareFloat(s, AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset));
+ case DOUBLE:
+ return compareDouble(s, ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset));
+ default:
+ throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_FLOAT_TYPE_TAG,
+ typeTag2.serialize());
}
}
@@ -278,33 +243,21 @@
long s = AInt64SerializerDeserializer.getLong(leftBytes, leftOffset);
switch (typeTag2) {
- case INT8: {
- byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightOffset);
- return compareLong(s, v2);
- }
- case INT16: {
- short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightOffset);
- return compareLong(s, v2);
- }
- case INT32: {
- int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightOffset);
- return compareLong(s, v2);
- }
- case INT64: {
- long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightOffset);
- return compareLong(s, v2);
- }
- case FLOAT: {
- float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset);
- return compareFloat(s, v2);
- }
- case DOUBLE: {
- double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset);
- return compareDouble(s, v2);
- }
- default: {
- throw new HyracksDataException("Comparison is undefined between types AInt64 and " + typeTag2 + " .");
- }
+ case INT8:
+ return compareLong(s, AInt8SerializerDeserializer.getByte(rightBytes, rightOffset));
+ case INT16:
+ return compareLong(s, AInt16SerializerDeserializer.getShort(rightBytes, rightOffset));
+ case INT32:
+ return compareLong(s, AInt32SerializerDeserializer.getInt(rightBytes, rightOffset));
+ case INT64:
+ return compareLong(s, AInt64SerializerDeserializer.getLong(rightBytes, rightOffset));
+ case FLOAT:
+ return compareFloat(s, AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset));
+ case DOUBLE:
+ return compareDouble(s, ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset));
+ default:
+ throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
+ typeTag2.serialize());
}
}
@@ -340,9 +293,9 @@
double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset);
return compareDouble(s, v2);
}
- default: {
- throw new HyracksDataException("Comparison is undefined between types AInt32 and " + typeTag2 + " .");
- }
+ default:
+ throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_INT32_TYPE_TAG,
+ typeTag2.serialize());
}
}
@@ -379,7 +332,8 @@
return compareDouble(s, v2);
}
default: {
- throw new HyracksDataException("Comparison is undefined between types AInt16 and " + typeTag2 + " .");
+ throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+ typeTag2.serialize());
}
}
}
@@ -392,33 +346,21 @@
byte s = AInt8SerializerDeserializer.getByte(leftBytes, leftStart);
switch (typeTag2) {
- case INT8: {
- byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightStart);
- return compareByte(s, v2);
- }
- case INT16: {
- short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightStart);
- return compareShort(s, v2);
- }
- case INT32: {
- int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightStart);
- return compareInt(s, v2);
- }
- case INT64: {
- long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightStart);
- return compareLong(s, v2);
- }
- case FLOAT: {
- float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightStart);
- return compareFloat(s, v2);
- }
- case DOUBLE: {
- double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightStart);
- return compareDouble(s, v2);
- }
- default: {
- throw new HyracksDataException("Comparison is undefined between types AInt16 and " + typeTag2 + " .");
- }
+ case INT8:
+ return compareByte(s, AInt8SerializerDeserializer.getByte(rightBytes, rightStart));
+ case INT16:
+ return compareShort(s, AInt16SerializerDeserializer.getShort(rightBytes, rightStart));
+ case INT32:
+ return compareInt(s, AInt32SerializerDeserializer.getInt(rightBytes, rightStart));
+ case INT64:
+ return compareLong(s, AInt64SerializerDeserializer.getLong(rightBytes, rightStart));
+ case FLOAT:
+ return compareFloat(s, AFloatSerializerDeserializer.getFloat(rightBytes, rightStart));
+ case DOUBLE:
+ return compareDouble(s, ADoubleSerializerDeserializer.getDouble(rightBytes, rightStart));
+ default:
+ throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_INT8_TYPE_TAG,
+ typeTag2.serialize());
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/DeepEqualAssessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/DeepEqualAssessor.java
index b4e0375..0c397f0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/DeepEqualAssessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/DeepEqualAssessor.java
@@ -25,7 +25,6 @@
import org.apache.asterix.om.types.hierachy.ATypeHierarchy.Domain;
import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
import org.apache.asterix.runtime.evaluators.visitors.DeepEqualityVisitor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -39,11 +38,12 @@
*/
public class DeepEqualAssessor {
+ private static final String DEEP_EQUAL = "deep-equal";
+ private static final double EPSILON = 1E-10;
private final DeepEqualityVisitor equalityVisitor = new DeepEqualityVisitor();
public boolean isEqual(IVisitablePointable leftPointable, IVisitablePointable rightPointable)
- throws AlgebricksException, AsterixException {
-
+ throws HyracksDataException, AsterixException {
if (leftPointable == null || rightPointable == null) {
return false;
}
@@ -59,16 +59,11 @@
// If types are numeric compare their real values instead
if (ATypeHierarchy.isSameTypeDomain(leftTypeTag, rightTypeTag, false)
&& ATypeHierarchy.getTypeDomain(leftTypeTag) == Domain.NUMERIC) {
- try {
- double leftVal = ATypeHierarchy.getDoubleValue(leftPointable.getByteArray(),
+ double leftVal = ATypeHierarchy.getDoubleValue(DEEP_EQUAL, 0, leftPointable.getByteArray(),
leftPointable.getStartOffset());
- double rightVal = ATypeHierarchy.getDoubleValue(rightPointable.getByteArray(),
+ double rightVal = ATypeHierarchy.getDoubleValue(DEEP_EQUAL, 1, rightPointable.getByteArray(),
rightPointable.getStartOffset());
- return (leftVal == rightVal);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
-
+ return Math.abs(leftVal - rightVal) < EPSILON;
} else {
return false;
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
index 122c1a2..dd2f5bd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class EqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
@@ -44,12 +44,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
@Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
index 1bc731a..2650f00 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class GreaterThanDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
@@ -44,12 +44,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
@Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
index b6318ad..073a7c2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class GreaterThanOrEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
@@ -44,12 +44,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
@Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
index d375a61..b2121e7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class LessThanDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
@@ -44,12 +44,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
@Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
index 4bcaa5b..1b70ee2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class LessThanOrEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
@@ -44,12 +44,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
@Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
index 11ee60f..de100f1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class NotEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
@@ -44,12 +44,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
@Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringConstructorDescriptor.java
index 2d1155b..5017c09 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringConstructorDescriptor.java
@@ -22,11 +22,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.parsers.ByteArrayBase64ParserFactory;
public class ABinaryBase64StringConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -39,13 +39,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new ABinaryHexStringConstructorDescriptor.ABinaryConstructorEvaluator(args[0],
ByteArrayBase64ParserFactory.INSTANCE, ctx);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java
index 74f4b65..920b47a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java
@@ -27,11 +27,13 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -51,13 +53,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new ABinaryConstructorEvaluator(args[0], ByteArrayHexParserFactory.INSTANCE, ctx);
}
};
@@ -77,23 +78,22 @@
private UTF8StringPointable utf8Ptr = new UTF8StringPointable();
public ABinaryConstructorEvaluator(IScalarEvaluatorFactory copyEvaluatorFactory,
- IValueParserFactory valueParserFactory, IHyracksTaskContext context) throws AlgebricksException {
+ IValueParserFactory valueParserFactory, IHyracksTaskContext context) throws HyracksDataException {
eval = copyEvaluatorFactory.createScalarEvaluator(context);
byteArrayParser = valueParserFactory.createValueParser();
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
eval.evaluate(tuple, inputArg);
byte[] binary = inputArg.getByteArray();
int startOffset = inputArg.getStartOffset();
int len = inputArg.getLength();
- ATypeTag tt = ATypeTag.VALUE_TYPE_MAPPING[binary[startOffset]];
- if (tt == ATypeTag.BINARY) {
+ if (binary[startOffset] == ATypeTag.SERIALIZED_BINARY_TYPE_TAG) {
result.set(inputArg);
- } else if (tt == ATypeTag.STRING) {
+ } else if (binary[startOffset] == ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
resultStorage.reset();
utf8Ptr.set(inputArg.getByteArray(), startOffset + 1, len - 1);
char[] buffer = utf8Ptr.toString().toCharArray();
@@ -101,10 +101,13 @@
byteArrayParser.parse(buffer, 0, buffer.length, out);
result.set(resultStorage);
} else {
- throw new AlgebricksException("binary type of " + tt + "haven't implemented yet.");
+ throw new TypeMismatchException(AsterixBuiltinFunctions.BINARY_HEX_CONSTRUCTOR, 0,
+ binary[startOffset], ATypeTag.SERIALIZED_BINARY_TYPE_TAG,
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new InvalidDataFormatException(AsterixBuiltinFunctions.BINARY_HEX_CONSTRUCTOR, e,
+ ATypeTag.SERIALIZED_BINARY_TYPE_TAG);
}
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java
index 85ecb75..9eb0563 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java
@@ -30,13 +30,15 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -58,13 +60,12 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
private IPointable inputArg = new VoidPointable();
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of boolean";
private final byte[] TRUE = UTF8StringUtil.writeStringToBytes("true");
private final byte[] FALSE = UTF8StringUtil.writeStringToBytes("false");
IBinaryComparator utf8BinaryComparator = AqlBinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE
@@ -74,7 +75,7 @@
.getSerializerDeserializer(BuiltinType.ABOOLEAN);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -94,13 +95,16 @@
result.set(resultStorage);
return;
} else {
- throw new AlgebricksException(errorMessage);
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
}
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[startOffset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java
index 319ab0e..07aed4a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java
@@ -31,12 +31,14 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -59,13 +61,12 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
private final IPointable inputArg = new VoidPointable();
private final IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private final String errorMessage = "This can not be an instance of circle";
private final AMutablePoint aPoint = new AMutablePoint(0, 0);
private AMutableCircle aCircle = new AMutableCircle(null, 0);
@SuppressWarnings("unchecked")
@@ -75,7 +76,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -94,11 +95,13 @@
aCircle.setValue(aPoint, Double.parseDouble(s.substring(spaceIndex + 1, s.length())));
circleSerde.serialize(aCircle, out);
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
result.set(resultStorage);
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
index 06b9948..e22f076 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
@@ -32,12 +32,14 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -59,14 +61,13 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
private IPointable inputArg = new VoidPointable();
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of date";
private AMutableDate aDate = new AMutableDate(0);
@SuppressWarnings("unchecked")
private ISerializerDeserializer<ADate> dateSerde = AqlSerializerDeserializerProvider.INSTANCE
@@ -75,7 +76,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -89,9 +90,8 @@
// the string to be parsed should be at least 8 characters: YYYYMMDD
if (stringLength < 8) {
- throw new AlgebricksException(errorMessage
- + ": the string length should be at least 8 (YYYYMMDD) but it is "
- + stringLength);
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG);
}
int startOffset = utf8Ptr.getCharStartOffset();
@@ -113,11 +113,12 @@
aDate.setValue((int) (chrononTimeInMs / GregorianCalendarSystem.CHRONON_OF_DAY) - temp);
dateSerde.serialize(aDate, out);
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
result.set(resultStorage);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_DATE_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
index 63f3461..cb16578 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
@@ -32,12 +32,14 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -59,13 +61,12 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
private IPointable inputArg = new VoidPointable();
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of datetime";
private AMutableDateTime aDateTime = new AMutableDateTime(0L);
@SuppressWarnings("unchecked")
private ISerializerDeserializer<ADateTime> datetimeSerde = AqlSerializerDeserializerProvider.INSTANCE
@@ -73,7 +74,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -87,9 +88,8 @@
int startOffset = utf8Ptr.getCharStartOffset();
// the string to be parsed should be at least 14 characters: YYYYMMDDhhmmss
if (stringLength < 14) {
- throw new AlgebricksException(errorMessage
- + ": the string length should be at least 14 (YYYYMMDDhhmmss) but it is "
- + stringLength);
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
// +1 if it is negative (-)
short timeOffset = (short) ((serString[startOffset] == '-') ? 1 : 0);
@@ -99,7 +99,8 @@
if (serString[startOffset + timeOffset] != 'T') {
timeOffset += 2;
if (serString[startOffset + timeOffset] != 'T') {
- throw new AlgebricksException(errorMessage + ": missing T");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
}
@@ -112,11 +113,13 @@
aDateTime.setValue(chrononTimeInMs);
datetimeSerde.serialize(aDateTime, out);
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
result.set(resultStorage);
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java
index f95a441..f8e34de 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java
@@ -31,12 +31,14 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -60,14 +62,13 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
private IPointable inputArg = new VoidPointable();
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of day-time-duration";
private AMutableDayTimeDuration aDayTimeDuration = new AMutableDayTimeDuration(0);
@SuppressWarnings("unchecked")
private ISerializerDeserializer<ADayTimeDuration> dayTimeDurationSerde = AqlSerializerDeserializerProvider.INSTANCE
@@ -75,7 +76,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -93,11 +94,13 @@
dayTimeDurationSerde.serialize(aDayTimeDuration, out);
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
result.set(resultStorage);
- } catch (Exception e1) {
- throw new AlgebricksException(e1);
+ } catch (Exception e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java
index 92d7c72..41fade5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java
@@ -31,13 +31,15 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -60,13 +62,12 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput out = resultStorage.getDataOutput();
private final IPointable inputArg = new VoidPointable();
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of double";
private final byte[] POSITIVE_INF = UTF8StringUtil.writeStringToBytes("INF");
private final byte[] NEGATIVE_INF = UTF8StringUtil.writeStringToBytes("-INF");
private final byte[] NAN = UTF8StringUtil.writeStringToBytes("NaN");
@@ -80,7 +81,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -104,11 +105,13 @@
}
doubleSerde.serialize(aDouble, out);
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
result.set(resultStorage);
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java
index 18dc471..0ca4177 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java
@@ -19,6 +19,7 @@
package org.apache.asterix.runtime.evaluators.constructors;
import java.io.DataOutput;
+import java.io.IOException;
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.ADuration;
@@ -31,12 +32,14 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -58,13 +61,12 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
private IPointable inputArg = new VoidPointable();
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of duration";
private AMutableDuration aDuration = new AMutableDuration(0, 0);
@SuppressWarnings("unchecked")
private ISerializerDeserializer<ADuration> durationSerde = AqlSerializerDeserializerProvider.INSTANCE
@@ -72,7 +74,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -87,11 +89,13 @@
stringLength, aDuration, ADurationParseOption.All);
durationSerde.serialize(aDuration, out);
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
result.set(resultStorage);
- } catch (Exception e1) {
- throw new AlgebricksException(e1);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java
index e293552..c514002 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java
@@ -31,13 +31,15 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -60,27 +62,17 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
private IPointable inputArg = new VoidPointable();
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of float";
private final byte[] POSITIVE_INF = UTF8StringUtil.writeStringToBytes("INF");
private final byte[] NEGATIVE_INF = UTF8StringUtil.writeStringToBytes("-INF");
private final byte[] NAN = UTF8StringUtil.writeStringToBytes("NaN");
-
- // private int offset = 3, value = 0, pointIndex = 0, eIndex
- // = 1;
- // private int integerPart = 0, fractionPart = 0,
- // exponentPart = 0;
- // float floatValue = 0;
- // boolean positiveInteger = true, positiveExponent = true,
- // expectingInteger = true,
- // expectingFraction = false, expectingExponent = false;
- IBinaryComparator utf8BinaryComparator = AqlBinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE
- .createBinaryComparator();
+ private IBinaryComparator utf8BinaryComparator = AqlBinaryComparatorFactoryProvider.
+ UTF8STRING_POINTABLE_INSTANCE.createBinaryComparator();
private AMutableFloat aFloat = new AMutableFloat(0);
@SuppressWarnings("unchecked")
private ISerializerDeserializer<AFloat> floatSerde = AqlSerializerDeserializerProvider.INSTANCE
@@ -88,7 +80,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -112,11 +104,13 @@
}
floatSerde.serialize(aFloat, out);
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
result.set(resultStorage);
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_FLOAT_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java
index f413c82..c64dc5b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java
@@ -30,12 +30,14 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -58,7 +60,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
@@ -67,7 +69,6 @@
private short value;
private int offset;
private boolean positive;
- private String errorMessage = "This can not be an instance of int16";
private AMutableInt16 aInt16 = new AMutableInt16((short) 0);
@SuppressWarnings("unchecked")
private ISerializerDeserializer<AInt16> int16Serde = AqlSerializerDeserializerProvider.INSTANCE
@@ -75,7 +76,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -102,11 +103,13 @@
&& serString[offset + 2] == '6' && offset + 3 == end) {
break;
} else {
- throw new AlgebricksException(errorMessage);
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INT16_TYPE_TAG);
}
}
if (value < 0) {
- throw new AlgebricksException(errorMessage);
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INT16_TYPE_TAG);
}
if (value > 0 && !positive) {
value *= -1;
@@ -115,11 +118,13 @@
aInt16.setValue(value);
int16Serde.serialize(aInt16, out);
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
result.set(resultStorage);
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_INT16_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java
index a727026..533e49e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java
@@ -30,12 +30,14 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -57,7 +59,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -66,7 +68,6 @@
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
private int value, offset;
private boolean positive;
- private String errorMessage = "This can not be an instance of int32";
private AMutableInt32 aInt32 = new AMutableInt32(0);
@SuppressWarnings("unchecked")
private ISerializerDeserializer<AInt32> int32Serde = AqlSerializerDeserializerProvider.INSTANCE
@@ -74,7 +75,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -101,11 +102,13 @@
&& serString[offset + 2] == '2' && offset + 3 == end) {
break;
} else {
- throw new AlgebricksException(errorMessage);
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG);
}
}
if (value < 0) {
- throw new AlgebricksException(errorMessage);
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG);
}
if (value > 0 && !positive) {
value *= -1;
@@ -114,11 +117,13 @@
aInt32.setValue(value);
int32Serde.serialize(aInt32, out);
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
result.set(resultStorage);
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java
index 12775fb..b380665 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java
@@ -30,12 +30,13 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -57,7 +58,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -67,7 +68,6 @@
private long value;
private int offset;
private boolean positive;
- private String errorMessage = "This can not be an instance of int64";
private AMutableInt64 aInt64 = new AMutableInt64(0);
@SuppressWarnings("unchecked")
private ISerializerDeserializer<AInt64> int64Serde = AqlSerializerDeserializerProvider.INSTANCE
@@ -75,7 +75,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -102,11 +102,13 @@
&& serString[offset + 2] == '4' && offset + 3 == end) {
break;
} else {
- throw new AlgebricksException(errorMessage);
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INT64_TYPE_TAG);
}
}
- if (value < 0 && value != -9223372036854775808L) {
- throw new AlgebricksException(errorMessage);
+ if (value < 0 && value != Long.MIN_VALUE) {
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INT64_TYPE_TAG);
}
if (value > 0 && !positive) {
value *= -1;
@@ -115,11 +117,13 @@
aInt64.setValue(value);
int64Serde.serialize(aInt64, out);
} else {
- throw new AlgebricksException(errorMessage);
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INT64_TYPE_TAG);
}
result.set(resultStorage);
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_INT64_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java
index 90359de..5b9f918 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java
@@ -30,12 +30,13 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -57,7 +58,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -67,7 +68,6 @@
private byte value;
private int offset;
private boolean positive;
- private String errorMessage = "This can not be an instance of int8";
private AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
@SuppressWarnings("unchecked")
private ISerializerDeserializer<AInt8> int8Serde = AqlSerializerDeserializerProvider.INSTANCE
@@ -75,7 +75,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -102,11 +102,13 @@
&& offset + 2 == end) {
break;
} else {
- throw new AlgebricksException(errorMessage);
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INT8_TYPE_TAG);
}
}
if (value < 0) {
- throw new AlgebricksException(errorMessage);
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INT8_TYPE_TAG);
}
if (value > 0 && !positive) {
value *= -1;
@@ -115,11 +117,13 @@
aInt8.setValue(value);
int8Serde.serialize(aInt8, out);
} else {
- throw new AlgebricksException(errorMessage);
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INT8_TYPE_TAG);
}
result.set(resultStorage);
} catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
+ throw new InvalidDataFormatException(getIdentifier(), e1,
+ ATypeTag.SERIALIZED_INT8_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java
index 58a65e4..763f409 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java
@@ -34,12 +34,15 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -61,7 +64,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
@@ -69,14 +72,13 @@
private IPointable argPtr1 = new VoidPointable();
private IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
private IScalarEvaluator eval1 = args[1].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of interval (only support Date/Time/Datetime)";
private AMutableInterval aInterval = new AMutableInterval(0L, 0L, (byte) 0);
@SuppressWarnings("unchecked")
private ISerializerDeserializer<AInterval> intervalSerde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.AINTERVAL);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -88,13 +90,10 @@
try {
if (bytes0[offset0] != bytes1[offset1]) {
- throw new AlgebricksException(FID.getName()
- + ": expects both arguments to be of the same type. Either DATE/TIME/DATETIME, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + " and "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]));
+ throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0], bytes1[offset1]);
}
- long intervalStart = 0, intervalEnd = 0;
+ long intervalStart, intervalEnd;
ATypeTag intervalType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
switch (intervalType) {
@@ -111,20 +110,19 @@
intervalEnd = ADateTimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
break;
default:
- throw new AlgebricksException(
- FID.getName() + ": expects NULL/DATE/TIME/DATETIME as arguments, but got "
- + intervalType);
+ throw new UnsupportedItemTypeException(getIdentifier(), bytes0[offset0]);
}
if (intervalEnd < intervalStart) {
- throw new AlgebricksException(
- FID.getName() + ": interval end must not be less than the interval start.");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
aInterval.setValue(intervalStart, intervalEnd, intervalType.serialize());
intervalSerde.serialize(aInterval, out);
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java
index ca115fb..67966f6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java
@@ -31,22 +31,23 @@
import org.apache.asterix.om.base.AMutableInterval;
import org.apache.asterix.om.base.temporal.ADateParserFactory;
import org.apache.asterix.om.base.temporal.ADurationParserFactory;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
import org.apache.asterix.om.base.temporal.DurationArithmeticOperations;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -70,7 +71,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -79,7 +80,6 @@
private IPointable argPtr1 = new VoidPointable();
private IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
private IScalarEvaluator eval1 = args[1].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of interval (from Date)";
private AMutableInterval aInterval = new AMutableInterval(0L, 0L, (byte) 0);
private AMutableDuration aDuration = new AMutableDuration(0, 0L);
@@ -89,7 +89,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -113,9 +113,8 @@
int startOffset = utf8Ptr.getCharStartOffset();
intervalStart = ADateParserFactory.parseDatePart(bytes0, startOffset, stringLength);
} else {
- throw new AlgebricksException(
- FID.getName() + ": expects NULL/STRING/DATE for the first argument, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]));
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
if (bytes1[offset1] == ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
@@ -139,26 +138,23 @@
intervalEnd = DurationArithmeticOperations.addDuration(intervalStart,
aDuration.getMonths(), aDuration.getMilliseconds(), false);
} else {
- throw new AlgebricksException(FID.getName()
- + ": expects NULL/STRING/DURATION for the second argument, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]));
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
intervalStart = GregorianCalendarSystem.getChrononInDays(intervalStart);
intervalEnd = GregorianCalendarSystem.getChrononInDays(intervalEnd);
if (intervalEnd < intervalStart) {
- throw new AlgebricksException(
- FID.getName() + ": interval end must not be less than the interval start.");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
aInterval.setValue(intervalStart, intervalEnd, ATypeTag.SERIALIZED_DATE_TYPE_TAG);
intervalSerde.serialize(aInterval, out);
-
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
- } catch (Exception e2) {
- throw new AlgebricksException(e2);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java
index b8c56fb..030023e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java
@@ -31,22 +31,23 @@
import org.apache.asterix.om.base.AMutableInterval;
import org.apache.asterix.om.base.temporal.ADateParserFactory;
import org.apache.asterix.om.base.temporal.ADurationParserFactory;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
import org.apache.asterix.om.base.temporal.ATimeParserFactory;
import org.apache.asterix.om.base.temporal.DurationArithmeticOperations;
+import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -69,7 +70,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
@@ -77,7 +78,6 @@
private IPointable argPtr1 = new VoidPointable();
private IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
private IScalarEvaluator eval1 = args[1].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of interval (from Date)";
private AMutableInterval aInterval = new AMutableInterval(0L, 0L, (byte) 0);
private AMutableDuration aDuration = new AMutableDuration(0, 0L);
@@ -87,7 +87,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -116,7 +116,8 @@
if (bytes0[startOffset + timeOffset] != 'T') {
timeOffset += 2;
if (argPtr0.getByteArray()[startOffset + timeOffset] != 'T') {
- throw new AlgebricksException(errorMessage + ": missing T");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
}
@@ -124,9 +125,8 @@
intervalStart += ATimeParserFactory.parseTimePart(bytes0, startOffset + timeOffset + 1,
stringLength - timeOffset - 1);
} else {
- throw new AlgebricksException(
- FID.getName() + ": expects NULL/STRING/DATETIME for the first argument but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]));
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
if (bytes1[offset1] == ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
@@ -149,23 +149,20 @@
intervalEnd = DurationArithmeticOperations.addDuration(intervalStart,
aDuration.getMonths(), aDuration.getMilliseconds(), false);
} else {
- throw new AlgebricksException(FID.getName()
- + ": expects NULL/STRING/DURATION for the second argument but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]));
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
if (intervalEnd < intervalStart) {
- throw new AlgebricksException(
- FID.getName() + ": interval end must not be less than the interval start.");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
aInterval.setValue(intervalStart, intervalEnd, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
intervalSerde.serialize(aInterval, out);
-
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
- } catch (Exception e2) {
- throw new AlgebricksException(e2);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java
index 49644f3..7fa0514 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java
@@ -29,23 +29,24 @@
import org.apache.asterix.om.base.AMutableDuration;
import org.apache.asterix.om.base.AMutableInterval;
import org.apache.asterix.om.base.temporal.ADurationParserFactory;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
import org.apache.asterix.om.base.temporal.ATimeParserFactory;
import org.apache.asterix.om.base.temporal.DurationArithmeticOperations;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -68,7 +69,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -77,7 +78,6 @@
private IPointable argPtr1 = new VoidPointable();
private IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
private IScalarEvaluator eval1 = args[1].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of interval (from Date)";
private AMutableInterval aInterval = new AMutableInterval(0L, 0L, (byte) 0);
private AMutableDuration aDuration = new AMutableDuration(0, 0L);
@@ -87,7 +87,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -111,9 +111,8 @@
intervalStart = ATimeParserFactory.parseTimePart(bytes0, utf8Ptr.getCharStartOffset(),
stringLength);
} else {
- throw new AlgebricksException(
- FID.getName() + ": expects NULL/STRING/TIME for the first argument, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]));
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_TIME_TYPE_TAG, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
if (intervalStart < 0) {
@@ -122,8 +121,8 @@
if (bytes1[offset1] == ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
if (ADurationSerializerDeserializer.getYearMonth(bytes1, offset1 + 1) != 0) {
- throw new AlgebricksException(
- FID.getName() + ": cannot add a year-month duration to a time value.");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
intervalEnd = DurationArithmeticOperations.addDuration(intervalStart, 0,
@@ -138,14 +137,17 @@
ADurationParserFactory.parseDuration(bytes1, utf8Ptr.getCharStartOffset(), stringLength,
aDuration, ADurationParseOption.All);
if (aDuration.getMonths() != 0) {
- throw new AlgebricksException(
- FID.getName() + ": cannot add a year-month duration to a time value.");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
intervalEnd = DurationArithmeticOperations.addDuration(intervalStart, 0,
aDuration.getMilliseconds(), false);
} else {
- throw new AlgebricksException("Wrong format for interval constructor from time.");
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
if (intervalEnd > GregorianCalendarSystem.CHRONON_OF_DAY) {
@@ -153,17 +155,15 @@
}
if (intervalEnd < intervalStart) {
- throw new AlgebricksException(
- FID.getName() + ": interval end must not be less than the interval start.");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
aInterval.setValue(intervalStart, intervalEnd, ATypeTag.SERIALIZED_TIME_TYPE_TAG);
intervalSerde.serialize(aInterval, out);
-
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
- } catch (Exception e2) {
- throw new AlgebricksException(e2);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java
index b912a4e..c3123da 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java
@@ -31,12 +31,14 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -59,14 +61,13 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
private IPointable inputArg = new VoidPointable();
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of line";
private AMutableLine aLine = new AMutableLine(null, null);
private AMutablePoint[] aPoint = { new AMutablePoint(0, 0), new AMutablePoint(0, 0) };
@SuppressWarnings("unchecked")
@@ -75,7 +76,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -96,11 +97,13 @@
aLine.setValue(aPoint[0], aPoint[1]);
lineSerde.serialize(aLine, out);
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
result.set(resultStorage);
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ANullConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ANullConstructorDescriptor.java
index 71359a8..e45efa9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ANullConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ANullConstructorDescriptor.java
@@ -30,13 +30,15 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -58,14 +60,13 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
private IPointable inputArg = new VoidPointable();
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of null";
private final byte[] NULL = UTF8StringUtil.writeStringToBytes("null");
IBinaryComparator utf8BinaryComparator = AqlBinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE
.createBinaryComparator();
@@ -74,7 +75,7 @@
.getSerializerDeserializer(BuiltinType.AMISSING);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
eval.evaluate(tuple, inputArg);
byte[] serString = inputArg.getByteArray();
@@ -89,13 +90,15 @@
result.set(resultStorage);
return;
} else {
- throw new AlgebricksException(errorMessage);
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_NULL_TYPE_TAG);
}
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_NULL_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java
index b7982da..a2a77cc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java
@@ -30,12 +30,14 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -58,14 +60,13 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
private IPointable inputArg = new VoidPointable();
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of point3d";
private AMutablePoint3D aPoint3D = new AMutablePoint3D(0, 0, 0);
@SuppressWarnings("unchecked")
private ISerializerDeserializer<APoint3D> point3DSerde = AqlSerializerDeserializerProvider.INSTANCE
@@ -73,7 +74,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -91,11 +92,13 @@
Double.parseDouble(s.substring(secondCommaIndex + 1, s.length())));
point3DSerde.serialize(aPoint3D, out);
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
result.set(resultStorage);
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java
index 78ac2da..c6f1f7b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java
@@ -30,12 +30,14 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -57,14 +59,13 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
private IPointable inputArg = new VoidPointable();
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of point";
private AMutablePoint aPoint = new AMutablePoint(0, 0);
@SuppressWarnings("unchecked")
private ISerializerDeserializer<APoint> pointSerde = AqlSerializerDeserializerProvider.INSTANCE
@@ -72,7 +73,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
@@ -88,11 +89,13 @@
Double.parseDouble(s.substring(s.indexOf(',') + 1, s.length())));
pointSerde.serialize(aPoint, out);
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
result.set(resultStorage);
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java
index 82e2157..3143b3f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java
@@ -28,11 +28,13 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -58,19 +60,17 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
private IPointable inputArg = new VoidPointable();
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of polygon";
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
-
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -83,7 +83,8 @@
String s = utf8Ptr.toString();
String[] points = WS.split(s.trim());
if (points.length <= 2) {
- throw new AlgebricksException(errorMessage);
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
}
out.writeByte(ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
out.writeShort(points.length);
@@ -93,11 +94,13 @@
Double.parseDouble(point[1]), out);
}
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
result.set(resultStorage);
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java
index e6eb5c1..41419dc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java
@@ -31,12 +31,14 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -58,14 +60,13 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
private IPointable inputArg = new VoidPointable();
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of rectangle";
private AMutableRectangle aRectangle = new AMutableRectangle(null, null);
private AMutablePoint[] aPoint = { new AMutablePoint(0, 0), new AMutablePoint(0, 0) };
@SuppressWarnings("unchecked")
@@ -74,8 +75,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
-
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -98,16 +98,18 @@
} else if (aPoint[0].getX() < aPoint[1].getX() && aPoint[0].getY() < aPoint[1].getY()) {
aRectangle.setValue(aPoint[0], aPoint[1]);
} else {
- throw new IllegalArgumentException(
- "Rectangle arugment must be either (bottom left point, top right point) or (top right point, bottom left point)");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
}
rectangle2DSerde.serialize(aRectangle, out);
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
result.set(resultStorage);
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java
index 930b584..2137b5d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java
@@ -33,11 +33,13 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -61,7 +63,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -72,7 +74,7 @@
private GrowableArray baaos = new GrowableArray();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
baaos.reset();
@@ -143,7 +145,7 @@
case UNORDEREDLIST:
case UUID:
default:
- throw new AlgebricksException("string of " + tt + " not supported");
+ throw new UnsupportedTypeException(getIdentifier(), serString[offset]);
}
builder.finish();
out.write(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
@@ -151,7 +153,8 @@
result.set(resultStorage);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
index a296ae9..2aeb6ee 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
@@ -32,12 +32,14 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -59,14 +61,13 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
private IPointable inputArg = new VoidPointable();
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of time";
private AMutableTime aTime = new AMutableTime(0);
@SuppressWarnings("unchecked")
private ISerializerDeserializer<ATime> timeSerde = AqlSerializerDeserializerProvider.INSTANCE
@@ -74,8 +75,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
-
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -90,9 +90,8 @@
// the string to be parsed should be at least 6 characters: hhmmss
if (stringLength < 6) {
- throw new AlgebricksException(errorMessage
- + ": the string length should be at least 6 (hhmmss) but it is "
- + stringLength);
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
}
int chrononTimeInMs = ATimeParserFactory.parseTimePart(serString, startOffset,
@@ -105,11 +104,13 @@
aTime.setValue(chrononTimeInMs);
timeSerde.serialize(aTime, out);
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
result.set(resultStorage);
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringConstructorDescriptor.java
index 44140fb..bddd9bc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringConstructorDescriptor.java
@@ -30,12 +30,14 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -62,14 +64,13 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
private IPointable inputArg = new VoidPointable();
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of UUID";
private AMutableUUID uuid = new AMutableUUID();
@SuppressWarnings("unchecked")
private ISerializerDeserializer<AUUID> uuidSerde = AqlSerializerDeserializerProvider.INSTANCE
@@ -78,7 +79,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -93,11 +94,12 @@
uuid.parseUUIDHexBytes(serString, offset);
uuidSerde.serialize(uuid, out);
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[start],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
result.set(resultStorage);
- } catch (IOException e1) {
- throw new AlgebricksException(errorMessage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_UUID_TYPE_TAG);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java
index 160043f..889af14 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java
@@ -31,12 +31,14 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -59,14 +61,13 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
private IPointable inputArg = new VoidPointable();
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private String errorMessage = "This can not be an instance of year-month-duration";
private AMutableYearMonthDuration aYearMonthDuration = new AMutableYearMonthDuration(0);
@SuppressWarnings("unchecked")
private ISerializerDeserializer<AYearMonthDuration> yearMonthDurationSerde = AqlSerializerDeserializerProvider.INSTANCE
@@ -74,7 +75,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -89,11 +90,13 @@
stringLength, aYearMonthDuration, ADurationParseOption.YEAR_MONTH);
yearMonthDurationSerde.serialize(aYearMonthDuration, out);
} else {
- throw new AlgebricksException(errorMessage);
+ throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
result.set(resultStorage);
- } catch (Exception e1) {
- throw new AlgebricksException(e1);
+ } catch (Exception e) {
+ throw new InvalidDataFormatException(getIdentifier(), e,
+ ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OpenRecordConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OpenRecordConstructorDescriptor.java
index 94f3b20..002e1a9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OpenRecordConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/OpenRecordConstructorDescriptor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -67,7 +66,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
int n = args.length / 2;
final IScalarEvaluator[] evalNames = new IScalarEvaluator[n];
final IScalarEvaluator[] evalFields = new IScalarEvaluator[n];
@@ -85,31 +84,27 @@
private boolean first = true;
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
- try {
- resultStorage.reset();
- closedFieldId = 0;
- if (first) {
- first = false;
- recBuilder.reset(recType);
- }
- recBuilder.init();
- for (int i = 0; i < evalFields.length; i++) {
- evalFields[i].evaluate(tuple, fieldValuePointable);
- boolean openField = openFields[i];
- if (openField) {
- evalNames[i].evaluate(tuple, fieldNamePointable);
- recBuilder.addField(fieldNamePointable, fieldValuePointable);
- } else {
- recBuilder.addField(closedFieldId, fieldValuePointable);
- closedFieldId++;
- }
- }
- recBuilder.write(out, true);
- result.set(resultStorage);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ closedFieldId = 0;
+ if (first) {
+ first = false;
+ recBuilder.reset(recType);
}
+ recBuilder.init();
+ for (int i = 0; i < evalFields.length; i++) {
+ evalFields[i].evaluate(tuple, fieldValuePointable);
+ boolean openField = openFields[i];
+ if (openField) {
+ evalNames[i].evaluate(tuple, fieldNamePointable);
+ recBuilder.addField(fieldNamePointable, fieldValuePointable);
+ } else {
+ recBuilder.addField(closedFieldId, fieldValuePointable);
+ closedFieldId++;
+ }
+ }
+ recBuilder.write(out, true);
+ result.set(resultStorage);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringBoolEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringBoolEval.java
index 99ba91a..128802d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringBoolEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringBoolEval.java
@@ -23,7 +23,7 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -39,7 +39,7 @@
.getSerializerDeserializer(BuiltinType.ABOOLEAN);
public AbstractBinaryStringBoolEval(IHyracksTaskContext context, IScalarEvaluatorFactory evalLeftFactory,
- IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID) throws AlgebricksException {
+ IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID) throws HyracksDataException {
super(context, evalLeftFactory, evalRightFactory, funcID);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringEval.java
index ab66ac6..7c0ee6b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringEval.java
@@ -22,13 +22,13 @@
import java.io.DataOutput;
import java.io.IOException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -55,7 +55,7 @@
private final FunctionIdentifier funcID;
public AbstractBinaryStringEval(IHyracksTaskContext context, IScalarEvaluatorFactory evalLeftFactory,
- IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID) throws AlgebricksException {
+ IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID) throws HyracksDataException {
this.evalLeft = evalLeftFactory.createScalarEvaluator(context);
this.evalRight = evalRightFactory.createScalarEvaluator(context);
this.funcID = funcID;
@@ -63,7 +63,7 @@
@SuppressWarnings("unchecked")
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable resultPointable) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable resultPointable) throws HyracksDataException {
resultStorage.reset();
// Gets the first argument.
@@ -79,11 +79,11 @@
int len1 = argPtrSecond.getLength();
// Type check.
- if (bytes0[offset0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG
- || bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(funcID.getName() + ": expects input type STRING, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + " and "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]) + ")!");
+ if (bytes0[offset0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(funcID, 0, bytes0[offset0], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(funcID, 1, bytes1[offset1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
// Sets StringUTF8Pointables.
@@ -94,7 +94,7 @@
try {
process(leftPtr, rightPtr, resultPointable);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringIntEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringIntEval.java
index cd7f6c4..4aecdb8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringIntEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringIntEval.java
@@ -24,7 +24,7 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.AMutableInt32;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -41,7 +41,7 @@
private AMutableInt32 resultValue = new AMutableInt32(0);
public AbstractBinaryStringIntEval(IHyracksTaskContext context, IScalarEvaluatorFactory evalLeftFactory,
- IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID) throws AlgebricksException {
+ IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID) throws HyracksDataException {
super(context, evalLeftFactory, evalRightFactory, funcID);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringStringEval.java
index a944de3..7cdf673 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringStringEval.java
@@ -22,10 +22,10 @@
import java.io.IOException;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.util.GrowableArray;
@@ -39,7 +39,7 @@
protected final UTF8StringBuilder resultBuilder = new UTF8StringBuilder();
public AbstractBinaryStringStringEval(IHyracksTaskContext context, IScalarEvaluatorFactory evalLeftFactory,
- IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID) throws AlgebricksException {
+ IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID) throws HyracksDataException {
super(context, evalLeftFactory, evalRightFactory, funcID);
}
@@ -60,7 +60,7 @@
* , the second input argument.
* @param resultStrPtr
* , a pointable that is supposed to point to the result string.
- * @throws AlgebricksException
+ * @throws HyracksDataException
*/
protected abstract void compute(UTF8StringPointable left, UTF8StringPointable right,
UTF8StringPointable resultStrPtr) throws IOException;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java
index 56b473e..0dbda8e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java
@@ -48,8 +48,12 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
+import org.apache.asterix.runtime.exceptions.OverflowException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.runtime.exceptions.UnderflowException;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -97,7 +101,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -134,156 +138,149 @@
@SuppressWarnings("unchecked")
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
- try {
- resultStorage.reset();
- resultType = 0;
- int currentType = 0;
- evalLeft.evaluate(tuple, argPtr0);
- evalRight.evaluate(tuple, argPtr1);
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ resultType = 0;
+ int currentType;
+ evalLeft.evaluate(tuple, argPtr0);
+ evalRight.evaluate(tuple, argPtr1);
- for (int i = 0; i < args.length; i++) {
- IPointable argPtr = i == 0 ? argPtr0 : argPtr1;
- byte[] bytes = argPtr.getByteArray();
- int offset = argPtr.getStartOffset();
+ for (int i = 0; i < args.length; i++) {
+ IPointable argPtr = i == 0 ? argPtr0 : argPtr1;
+ byte[] bytes = argPtr.getByteArray();
+ int offset = argPtr.getStartOffset();
- typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]);
- switch (typeTag) {
- case INT8: {
- currentType = typeInt8;
- operandsInteger[i] = AInt8SerializerDeserializer.getByte(bytes, offset + 1);
- operandsFloating[i] = operandsInteger[i];
- break;
- }
- case INT16: {
- currentType = typeInt16;
- operandsInteger[i] = AInt16SerializerDeserializer.getShort(bytes, offset + 1);
- operandsFloating[i] = operandsInteger[i];
- break;
- }
- case INT32: {
- currentType = typeInt32;
- operandsInteger[i] = AInt32SerializerDeserializer.getInt(bytes, offset + 1);
- operandsFloating[i] = operandsInteger[i];
- break;
- }
- case INT64: {
- currentType = typeInt64;
- operandsInteger[i] = AInt64SerializerDeserializer.getLong(bytes, offset + 1);
- operandsFloating[i] = operandsInteger[i];
- break;
- }
- case FLOAT: {
- currentType = typeFloat;
- operandsFloating[i] = AFloatSerializerDeserializer.getFloat(bytes, offset + 1);
- break;
- }
- case DOUBLE: {
- currentType = typeDouble;
- operandsFloating[i] = ADoubleSerializerDeserializer.getDouble(bytes,
- offset + 1);
- break;
- }
- case DATE:
- case TIME:
- case DATETIME:
- case DURATION:
- case YEARMONTHDURATION:
- case DAYTIMEDURATION:
- evaluateTemporalArthmeticOperation(typeTag);
- result.set(resultStorage);
- return;
- default: {
- throw new NotImplementedException(getIdentifier().getName()
- + (i == 0 ? ": Left" : ": Right")
- + " operand expects INT8/INT16/INT32/INT64/FLOAT/DOUBLE/NULL, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]));
- }
- }
-
- if (resultType < currentType) {
- resultType = currentType;
- }
+ typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]);
+ switch (typeTag) {
+ case INT8:
+ currentType = typeInt8;
+ operandsInteger[i] = AInt8SerializerDeserializer.getByte(bytes, offset + 1);
+ operandsFloating[i] = operandsInteger[i];
+ break;
+ case INT16:
+ currentType = typeInt16;
+ operandsInteger[i] = AInt16SerializerDeserializer.getShort(bytes, offset + 1);
+ operandsFloating[i] = operandsInteger[i];
+ break;
+ case INT32:
+ currentType = typeInt32;
+ operandsInteger[i] = AInt32SerializerDeserializer.getInt(bytes, offset + 1);
+ operandsFloating[i] = operandsInteger[i];
+ break;
+ case INT64:
+ currentType = typeInt64;
+ operandsInteger[i] = AInt64SerializerDeserializer.getLong(bytes, offset + 1);
+ operandsFloating[i] = operandsInteger[i];
+ break;
+ case FLOAT:
+ currentType = typeFloat;
+ operandsFloating[i] = AFloatSerializerDeserializer.getFloat(bytes, offset + 1);
+ break;
+ case DOUBLE:
+ currentType = typeDouble;
+ operandsFloating[i] = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1);
+ break;
+ case DATE:
+ case TIME:
+ case DATETIME:
+ case DURATION:
+ case YEARMONTHDURATION:
+ case DAYTIMEDURATION:
+ evaluateTemporalArthmeticOperation(typeTag);
+ result.set(resultStorage);
+ return;
+ default:
+ throw new TypeMismatchException(getIdentifier(), i, bytes[offset],
+ ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
+ ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG,
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_TIME_TYPE_TAG,
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG,
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG);
}
- long lres = 0;
- double dres = 0;
- switch (resultType) {
- case typeInt8:
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT8);
- lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
- if (lres > Byte.MAX_VALUE) {
- throw new AlgebricksException("Overflow happened.");
- }
- if (lres < Byte.MIN_VALUE) {
- throw new AlgebricksException("Underflow happened.");
- }
- aInt8.setValue((byte) lres);
- serde.serialize(aInt8, out);
- break;
- case typeInt16:
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT16);
- lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
- if (lres > Short.MAX_VALUE) {
- throw new AlgebricksException("Overflow happened.");
- }
- if (lres < Short.MIN_VALUE) {
- throw new AlgebricksException("Underflow happened.");
- }
- aInt16.setValue((short) lres);
- serde.serialize(aInt16, out);
- break;
- case typeInt32:
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT32);
- lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
- if (lres > Integer.MAX_VALUE) {
- throw new AlgebricksException("Overflow happened.");
- }
- if (lres < Integer.MIN_VALUE) {
- throw new AlgebricksException("Underflow happened.");
- }
- aInt32.setValue((int) lres);
- serde.serialize(aInt32, out);
- break;
- case typeInt64:
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT64);
- lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
- aInt64.setValue(lres);
- serde.serialize(aInt64, out);
- break;
- case typeFloat:
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AFLOAT);
- dres = evaluateDouble(operandsFloating[0], operandsFloating[1]);
- if (dres > Float.MAX_VALUE) {
- throw new AlgebricksException("Overflow happened.");
- }
- if (dres < -Float.MAX_VALUE) {
- throw new AlgebricksException("Underflow happened.");
- }
- aFloat.setValue((float) dres);
- serde.serialize(aFloat, out);
- break;
- case typeDouble:
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADOUBLE);
- aDouble.setValue(evaluateDouble(operandsFloating[0], operandsFloating[1]));
- serde.serialize(aDouble, out);
- break;
+ if (resultType < currentType) {
+ resultType = currentType;
}
- result.set(resultStorage);
- } catch (HyracksDataException hde) {
- throw new AlgebricksException(hde);
}
+
+ long lres;
+ double dres;
+ switch (resultType) {
+ case typeInt8:
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT8);
+ lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
+ if (lres > Byte.MAX_VALUE) {
+ throw new OverflowException(getIdentifier());
+ }
+ if (lres < Byte.MIN_VALUE) {
+ throw new UnderflowException(getIdentifier());
+ }
+ aInt8.setValue((byte) lres);
+ serde.serialize(aInt8, out);
+ break;
+ case typeInt16:
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT16);
+ lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
+ if (lres > Short.MAX_VALUE) {
+ throw new OverflowException(getIdentifier());
+ }
+ if (lres < Short.MIN_VALUE) {
+ throw new UnderflowException(getIdentifier());
+ }
+ aInt16.setValue((short) lres);
+ serde.serialize(aInt16, out);
+ break;
+ case typeInt32:
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT32);
+ lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
+ if (lres > Integer.MAX_VALUE) {
+ throw new OverflowException(getIdentifier());
+ }
+ if (lres < Integer.MIN_VALUE) {
+ throw new UnderflowException(getIdentifier());
+ }
+ aInt32.setValue((int) lres);
+ serde.serialize(aInt32, out);
+ break;
+ case typeInt64:
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT64);
+ lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
+ aInt64.setValue(lres);
+ serde.serialize(aInt64, out);
+ break;
+ case typeFloat:
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AFLOAT);
+ dres = evaluateDouble(operandsFloating[0], operandsFloating[1]);
+ if (dres > Float.MAX_VALUE) {
+ throw new OverflowException(getIdentifier());
+ }
+ if (dres < -Float.MAX_VALUE) {
+ throw new UnderflowException(getIdentifier());
+ }
+ aFloat.setValue((float) dres);
+ serde.serialize(aFloat, out);
+ break;
+ case typeDouble:
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADOUBLE);
+ aDouble.setValue(evaluateDouble(operandsFloating[0], operandsFloating[1]));
+ serde.serialize(aDouble, out);
+ break;
+ }
+ result.set(resultStorage);
}
@SuppressWarnings("unchecked")
private void evaluateTemporalArthmeticOperation(ATypeTag leftType)
- throws HyracksDataException, AlgebricksException {
+ throws HyracksDataException {
byte[] bytes1 = argPtr1.getByteArray();
int offset1 = argPtr1.getStartOffset();
ATypeTag rightType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
@@ -327,7 +324,7 @@
offset1 + 1);
break;
default:
- throw new NotImplementedException();
+ throw new UnsupportedTypeException(getIdentifier(), bytes1[offset1]);
}
dayTime = evaluateTimeInstanceArithmetic(leftChronon, rightChronon);
@@ -361,9 +358,8 @@
offset1 + 1);
break;
default:
- throw new NotImplementedException(getIdentifier().getName()
- + ": arithmetic operation between " + leftType + " and a "
- + rightType + " value is not supported.");
+ throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0],
+ bytes1[offset1]);
}
break;
case DATE:
@@ -394,9 +390,8 @@
offset1 + 1);
break;
default:
- throw new NotImplementedException(getIdentifier().getName()
- + ": arithmetic operation between " + leftType + " and a "
- + rightType + " value is not supported.");
+ throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0],
+ bytes1[offset1]);
}
break;
case YEARMONTHDURATION:
@@ -417,9 +412,8 @@
* GregorianCalendarSystem.CHRONON_OF_DAY;
break;
default:
- throw new NotImplementedException(getIdentifier().getName()
- + ": arithmetic operation between " + leftType + " and a "
- + rightType + " value is not supported.");
+ throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0],
+ bytes1[offset1]);
}
break;
case DURATION:
@@ -454,22 +448,19 @@
break;
}
default:
- throw new NotImplementedException(getIdentifier().getName()
- + ": arithmetic operation between " + leftType + " and a "
- + rightType + " value is not supported.");
+ throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0],
+ bytes1[offset1]);
}
break;
default:
- throw new NotImplementedException(
- getIdentifier().getName() + ": arithmetic operation between " + leftType
- + " and a " + rightType + " value is not supported.");
+ throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0],
+ bytes1[offset1]);
}
chronon = evaluateTimeDurationArithmetic(chronon, yearMonth, dayTime, isTimeOnly);
switch (resultType) {
case DATE:
-
if (chronon < 0 && chronon % GregorianCalendarSystem.CHRONON_OF_DAY != 0) {
chronon = chronon / GregorianCalendarSystem.CHRONON_OF_DAY - 1;
} else {
@@ -487,10 +478,8 @@
serde.serialize(aDatetime, out);
break;
default:
- throw new NotImplementedException(
- getIdentifier().getName() + ": arithmetic operation between " + leftType
- + " and a " + rightType + " value is not supported.");
-
+ throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0],
+ bytes1[offset1]);
}
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java
index 6eb6272..7814d90 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java
@@ -27,15 +27,15 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.AMutableString;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -69,7 +69,7 @@
public AbstractQuadStringStringEval(IHyracksTaskContext context, IScalarEvaluatorFactory eval0,
IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, IScalarEvaluatorFactory eval3,
- FunctionIdentifier funcID) throws AlgebricksException {
+ FunctionIdentifier funcID) throws HyracksDataException {
this.eval0 = eval0.createScalarEvaluator(context);
this.eval1 = eval1.createScalarEvaluator(context);
this.eval2 = eval2.createScalarEvaluator(context);
@@ -79,40 +79,53 @@
@SuppressWarnings("unchecked")
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval0.evaluate(tuple, array0);
eval1.evaluate(tuple, array1);
eval2.evaluate(tuple, array2);
eval3.evaluate(tuple, array3);
+ byte[] bytes0 = array0.getByteArray();
+ byte[] bytes1 = array1.getByteArray();
+ byte[] bytes2 = array2.getByteArray();
+ byte[] bytes3 = array3.getByteArray();
+
+ int start0 = array0.getStartOffset();
+ int start1 = array1.getStartOffset();
+ int start2 = array2.getStartOffset();
+ int start3 = array3.getStartOffset();
+
+ int len0 = array0.getLength();
+ int len1 = array1.getLength();
+ int len2 = array2.getLength();
+ int len3 = array3.getLength();
+
resultStorage.reset();
- if (array0.getByteArray()[array0.getStartOffset()] != ATypeTag.SERIALIZED_STRING_TYPE_TAG
- || array1.getByteArray()[array1.getStartOffset()] != ATypeTag.SERIALIZED_STRING_TYPE_TAG
- || array2.getByteArray()[array2.getStartOffset()] != ATypeTag.SERIALIZED_STRING_TYPE_TAG
- || array3.getByteArray()[array3.getStartOffset()] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(funcID.getName()
- + ": expects input type (STRING/NULL, STRING/NULL, STRING/NULL, STRING/NULL), but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(array0.getByteArray()[array0.getStartOffset()])
- + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(array1.getByteArray()[array1.getStartOffset()])
- + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(array2.getByteArray()[array2.getStartOffset()])
- + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(array3.getByteArray()[array3.getStartOffset()])
- + ".");
+ // Type check.
+ if (bytes0[start0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(funcID, 0, bytes0[start0], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+ if (bytes1[start1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(funcID, 1, bytes1[start1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+ if (bytes2[start2] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(funcID, 2, bytes2[start2], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+ if (bytes3[start3] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(funcID, 3, bytes1[start3], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
- strPtr1st.set(array0.getByteArray(), array0.getStartOffset() + 1, array0.getLength());
- strPtr2nd.set(array1.getByteArray(), array1.getStartOffset() + 1, array1.getLength());
- strPtr3rd.set(array2.getByteArray(), array2.getStartOffset() + 1, array2.getLength());
- strPtr4th.set(array3.getByteArray(), array3.getStartOffset() + 1, array3.getLength());
+ strPtr1st.set(bytes0, start0 + 1, len0);
+ strPtr2nd.set(bytes1, start1 + 1, len1);
+ strPtr3rd.set(bytes2, start2 + 1, len2);
+ strPtr4th.set(bytes3, start3 + 1, len3);
try {
String res = compute(strPtr1st, strPtr2nd, strPtr3rd, strPtr4th);
resultBuffer.setValue(res);
strSerde.serialize(resultBuffer, dout);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java
index 50e7135..a9196d5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java
@@ -23,11 +23,11 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
@@ -39,7 +39,7 @@
public AbstractTripleStringBoolEval(IHyracksTaskContext context, IScalarEvaluatorFactory eval0,
IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, FunctionIdentifier funcID)
- throws AlgebricksException {
+ throws HyracksDataException {
super(context, eval0, eval1, eval2, funcID);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringEval.java
index e54ac3f..d4a9329 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringEval.java
@@ -22,13 +22,13 @@
import java.io.DataOutput;
import java.io.IOException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -58,7 +58,7 @@
private final FunctionIdentifier funcID;
AbstractTripleStringEval(IHyracksTaskContext context, IScalarEvaluatorFactory eval0, IScalarEvaluatorFactory eval1,
- IScalarEvaluatorFactory eval2, FunctionIdentifier funcID) throws AlgebricksException {
+ IScalarEvaluatorFactory eval2, FunctionIdentifier funcID) throws HyracksDataException {
this.eval0 = eval0.createScalarEvaluator(context);
this.eval1 = eval1.createScalarEvaluator(context);
this.eval2 = eval2.createScalarEvaluator(context);
@@ -67,7 +67,7 @@
@SuppressWarnings("unchecked")
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
// Gets the first argument.
eval0.evaluate(tuple, argPtrFirst);
byte[] bytes0 = argPtrFirst.getByteArray();
@@ -87,13 +87,15 @@
int len2 = argPtrThird.getLength();
// Type check.
- if (bytes0[start0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG
- || bytes1[start1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG
- || bytes2[start2] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(funcID.getName() + ": expects iput type (STRING, STRING, STRING) but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[start0]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[start1]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes2[start2]) + ")");
+ // Type check.
+ if (bytes0[start0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(funcID, 0, bytes0[start0], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+ if (bytes1[start1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(funcID, 1, bytes1[start1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+ if (bytes2[start2] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(funcID, 2, bytes2[start2], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
// Sets argument UTF8Pointables.
@@ -107,7 +109,7 @@
try {
process(strPtr1st, strPtr2nd, strPtr3rd, result);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringIntEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringIntEval.java
index 97c843a..c7dc9dc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringIntEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringIntEval.java
@@ -24,11 +24,11 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.AMutableInt32;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
@@ -41,15 +41,14 @@
public AbstractTripleStringIntEval(IHyracksTaskContext context, IScalarEvaluatorFactory eval0,
IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, FunctionIdentifier funcID)
- throws AlgebricksException {
+ throws HyracksDataException {
super(context, eval0, eval1, eval2, funcID);
}
@SuppressWarnings("unchecked")
@Override
protected void process(UTF8StringPointable first, UTF8StringPointable second, UTF8StringPointable thrid,
- IPointable result)
- throws IOException {
+ IPointable result) throws IOException {
resultValue.setValue(compute(first, second, thrid));
intSerde.serialize(resultValue, dout);
result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java
index 6f6fdc4..f0a3678 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java
@@ -23,7 +23,7 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.AMutableString;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -40,7 +40,7 @@
public AbstractTripleStringStringEval(IHyracksTaskContext context, IScalarEvaluatorFactory eval0,
IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, FunctionIdentifier funcID)
- throws AlgebricksException {
+ throws HyracksDataException {
super(context, eval0, eval1, eval2, funcID);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericDoubleFunctionEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericDoubleFunctionEval.java
index 28e54d1..241452a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericDoubleFunctionEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericDoubleFunctionEval.java
@@ -19,7 +19,7 @@
package org.apache.asterix.runtime.evaluators.functions;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -31,35 +31,35 @@
abstract class AbstractUnaryNumericDoubleFunctionEval extends AbstractUnaryNumericFunctionEval {
public AbstractUnaryNumericDoubleFunctionEval(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory,
- FunctionIdentifier funcID) throws AlgebricksException {
+ FunctionIdentifier funcID) throws HyracksDataException {
super(context, argEvalFactory, funcID);
}
@Override
- protected void processInt8(byte arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt8(byte arg, IPointable resultPointable) throws HyracksDataException {
processDouble(arg, resultPointable);
}
@Override
- protected void processInt16(short arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt16(short arg, IPointable resultPointable) throws HyracksDataException {
processDouble(arg, resultPointable);
}
@Override
- protected void processInt32(int arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt32(int arg, IPointable resultPointable) throws HyracksDataException {
processDouble(arg, resultPointable);
}
@Override
- protected void processInt64(long arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt64(long arg, IPointable resultPointable) throws HyracksDataException {
processDouble(arg, resultPointable);
}
@Override
- protected void processFloat(float arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processFloat(float arg, IPointable resultPointable) throws HyracksDataException {
processDouble(arg, resultPointable);
}
@Override
- protected abstract void processDouble(double arg, IPointable resultPointable) throws AlgebricksException;
+ protected abstract void processDouble(double arg, IPointable resultPointable) throws HyracksDataException;
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericFunctionEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericFunctionEval.java
index 50ff0de..24c2135 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericFunctionEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericFunctionEval.java
@@ -38,13 +38,13 @@
import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -94,14 +94,14 @@
private final FunctionIdentifier funcID;
public AbstractUnaryNumericFunctionEval(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory,
- FunctionIdentifier funcID) throws AlgebricksException {
+ FunctionIdentifier funcID) throws HyracksDataException {
this.argEval = argEvalFactory.createScalarEvaluator(context);
this.funcID = funcID;
}
@SuppressWarnings("unchecked")
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
argEval.evaluate(tuple, argPtr);
byte[] data = argPtr.getByteArray();
@@ -126,8 +126,10 @@
double val = ADoubleSerializerDeserializer.getDouble(data, offset + 1);
processDouble(val, result);
} else {
- throw new AlgebricksException(funcID + " expects a numeric input type, but gets "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[offset]));
+ throw new TypeMismatchException(funcID, 0, data[offset], ATypeTag.SERIALIZED_INT8_TYPE_TAG,
+ ATypeTag.SERIALIZED_INT16_TYPE_TAG, ATypeTag.SERIALIZED_INT32_TYPE_TAG,
+ ATypeTag.SERIALIZED_INT64_TYPE_TAG, ATypeTag.SERIALIZED_FLOAT_TYPE_TAG,
+ ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
}
}
@@ -140,7 +142,7 @@
* ,
* the pointable that should be set to the result location.
*/
- protected abstract void processInt8(byte arg, IPointable resultPointable) throws AlgebricksException;
+ protected abstract void processInt8(byte arg, IPointable resultPointable) throws HyracksDataException;
/**
* Processes an int16 argument.
@@ -151,7 +153,7 @@
* ,
* the pointable that should be set to the result location.
*/
- protected abstract void processInt16(short arg, IPointable resultPointable) throws AlgebricksException;
+ protected abstract void processInt16(short arg, IPointable resultPointable) throws HyracksDataException;
/**
* Processes an int32 argument.
@@ -162,7 +164,7 @@
* ,
* the pointable that should be set to the result location.
*/
- protected abstract void processInt32(int arg, IPointable resultPointable) throws AlgebricksException;
+ protected abstract void processInt32(int arg, IPointable resultPointable) throws HyracksDataException;
/**
* Processes an int64 argument.
@@ -173,7 +175,7 @@
* ,
* the pointable that should be set to the result location.
*/
- protected abstract void processInt64(long arg, IPointable resultPointable) throws AlgebricksException;
+ protected abstract void processInt64(long arg, IPointable resultPointable) throws HyracksDataException;
/**
* Processes a float argument.
@@ -184,7 +186,7 @@
* ,
* the pointable that should be set to the result location.
*/
- protected abstract void processFloat(float arg, IPointable resultPointable) throws AlgebricksException;
+ protected abstract void processFloat(float arg, IPointable resultPointable) throws HyracksDataException;
/**
* Processes a double argument.
@@ -195,17 +197,17 @@
* ,
* the pointable that should be set to the result location.
*/
- protected abstract void processDouble(double arg, IPointable resultPointable) throws AlgebricksException;
+ protected abstract void processDouble(double arg, IPointable resultPointable) throws HyracksDataException;
// Serializes result into the result storage.
@SuppressWarnings("unchecked")
protected void serialize(IAObject result, ISerializerDeserializer serde, IPointable resultPointable)
- throws AlgebricksException {
+ throws HyracksDataException {
try {
serde.serialize(result, dataOutput);
resultPointable.set(resultStorage);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryStringStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryStringStringEval.java
index 8e68143..1f9909c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryStringStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryStringStringEval.java
@@ -22,13 +22,13 @@
import java.io.DataOutput;
import java.io.IOException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -52,22 +52,21 @@
private final FunctionIdentifier funcID;
AbstractUnaryStringStringEval(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory,
- FunctionIdentifier funcID) throws AlgebricksException {
+ FunctionIdentifier funcID) throws HyracksDataException {
this.argEval = argEvalFactory.createScalarEvaluator(context);
this.funcID = funcID;
}
@SuppressWarnings("unchecked")
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable resultPointable) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable resultPointable) throws HyracksDataException {
resultStorage.reset();
argEval.evaluate(tuple, argPtr);
byte[] argBytes = argPtr.getByteArray();
int offset = argPtr.getStartOffset();
byte inputTypeTag = argBytes[offset];
if (inputTypeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(funcID.getName() + ": expects input type to be STRING, but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(inputTypeTag) + ".");
+ throw new TypeMismatchException(funcID, 0, argBytes[offset], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
stringPtr.set(argBytes, offset + 1, argPtr.getLength() - 1);
resultArray.reset();
@@ -75,7 +74,7 @@
process(stringPtr, resultPointable);
writeResult(resultPointable);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java
index 7150517..1aba0ef 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java
@@ -31,7 +31,7 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -58,14 +58,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
-
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
final DataOutput out = resultStorage.getDataOutput();
final IPointable argPtr = new VoidPointable();
@@ -86,54 +84,54 @@
.getSerializerDeserializer(BuiltinType.AMISSING);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
- try {
- resultStorage.reset();
- int n = args.length;
- boolean res = true;
- boolean metNull = false;
- boolean metMissing = false;
- for (int i = 0; i < n; i++) {
- evals[i].evaluate(tuple, argPtr);
- byte[] bytes = argPtr.getByteArray();
- int offset = argPtr.getStartOffset();
- boolean isNull = false;
- boolean isMissing = false;
- if (bytes[offset] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
- isMissing = true;
- metMissing = true;
- }
- if (bytes[offset] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
- isNull = true;
- metNull = true;
- }
- if (isMissing || isNull) {
- continue;
- }
- boolean argResult = ABooleanSerializerDeserializer.getBoolean(bytes, offset + 1);
- if (argResult == false) {
- // anything AND FALSE = FALSE
- booleanSerde.serialize(ABoolean.FALSE, out);
- result.set(resultStorage);
- return;
- }
- res &= argResult;
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ int n = args.length;
+ boolean res = true;
+ boolean metNull = false;
+ boolean metMissing = false;
+ for (int i = 0; i < n; i++) {
+ evals[i].evaluate(tuple, argPtr);
+ byte[] bytes = argPtr.getByteArray();
+ int offset = argPtr.getStartOffset();
+ boolean isNull = false;
+ boolean isMissing = false;
+ if (bytes[offset] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
+ isMissing = true;
+ metMissing = true;
}
- if (metMissing) {
- // MISSING AND NULL = MISSING
- // MISSING AND TRUE = MISSING
- missingSerde.serialize(AMissing.MISSING, out);
- } else if (metNull) {
- // NULL AND TRUE = NULL
- nullSerde.serialize(ANull.NULL, out);
- } else {
- ABoolean aResult = res ? (ABoolean.TRUE) : (ABoolean.FALSE);
- booleanSerde.serialize(aResult, out);
+ if (bytes[offset] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
+ isNull = true;
+ metNull = true;
}
- result.set(resultStorage);
- } catch (HyracksDataException hde) {
- throw new AlgebricksException(hde);
+ if (isMissing || isNull) {
+ continue;
+ }
+ if (bytes[offset] != ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), i, bytes[offset],
+ ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
+ }
+ boolean argResult = ABooleanSerializerDeserializer.getBoolean(bytes, offset + 1);
+ if (argResult == false) {
+ // anything AND FALSE = FALSE
+ booleanSerde.serialize(ABoolean.FALSE, out);
+ result.set(resultStorage);
+ return;
+ }
+ res &= argResult;
}
+ if (metMissing) {
+ // MISSING AND NULL = MISSING
+ // MISSING AND TRUE = MISSING
+ missingSerde.serialize(AMissing.MISSING, out);
+ } else if (metNull) {
+ // NULL AND TRUE = NULL
+ nullSerde.serialize(ANull.NULL, out);
+ } else {
+ ABoolean aResult = res ? ABoolean.TRUE : ABoolean.FALSE;
+ booleanSerde.serialize(aResult, out);
+ }
+ result.set(resultStorage);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
index 16cec14..efb9b68 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
@@ -31,11 +31,12 @@
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.util.NonTaggedFormatUtil;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -75,7 +76,7 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -86,21 +87,21 @@
private int itemLength;
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+
+ resultStorage.reset();
+ evalList.evaluate(tuple, inputArgList);
+ byte[] serList = inputArgList.getByteArray();
+ int offset = inputArgList.getStartOffset();
+
+ if (serList[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
+ && serList[offset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
+ throw new TypeMismatchException(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER, 0,
+ serList[offset], ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
+ ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
+ }
try {
- resultStorage.reset();
- evalList.evaluate(tuple, inputArgList);
- byte[] serList = inputArgList.getByteArray();
- int offset = inputArgList.getStartOffset();
-
- if (serList[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
- && serList[offset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
- throw new AlgebricksException(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER.getName()
- + ": expects input type ORDEREDLIST/UNORDEREDLIST, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serList[offset]));
- }
-
if (serList[offset] == ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
if (AOrderedListSerializerDeserializer.getNumberOfItems(serList, offset) == 0) {
out.writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
@@ -136,9 +137,9 @@
result.set(resultStorage);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
} catch (AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/BinaryHashMap.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/BinaryHashMap.java
index 094e3cb..d89a63e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/BinaryHashMap.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/BinaryHashMap.java
@@ -109,7 +109,6 @@
* Retrieves value for given key. Returns null if key doesn't exist.
*
* @param key
- * @param value
* @return
* @throws HyracksDataException
*/
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
index 51b10dc..30dd7cc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
@@ -31,12 +31,12 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -86,7 +86,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new CastTypeEvaluator(reqType, inputType, recordEvalFactory.createScalarEvaluator(ctx));
}
};
@@ -106,26 +106,26 @@
private final Triple<IVisitablePointable, IAType, Boolean> arg;
public CastTypeEvaluator(IAType reqType, IAType inputType, IScalarEvaluator argEvaluator)
- throws AlgebricksException {
+ throws HyracksDataException {
try {
this.argEvaluator = argEvaluator;
this.inputPointable = allocatePointable(inputType, reqType);
this.resultPointable = allocatePointable(reqType, inputType);
this.arg = new Triple<>(resultPointable, reqType, Boolean.FALSE);
} catch (AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
argEvaluator.evaluate(tuple, argPointable);
inputPointable.set(argPointable);
inputPointable.accept(castVisitor, arg);
result.set(resultPointable);
} catch (Exception ioe) {
- throw new AlgebricksException(ioe);
+ throw new HyracksDataException(ioe);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java
index abd0d3a..8cef39e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java
@@ -24,11 +24,12 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -54,22 +55,20 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private IPointable inputArg = new VoidPointable();
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private String errorMessage = AsterixBuiltinFunctions.CHECK_UNKNOWN
- + ": the input value cannot be NULL.";
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, inputArg);
byte[] data = inputArg.getByteArray();
int offset = inputArg.getStartOffset();
if (data[offset] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG
|| data[offset] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
- throw new AlgebricksException(errorMessage);
+ throw new UnsupportedTypeException(getIdentifier(), data[offset]);
}
result.set(inputArg);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java
index 789d2c6..2831d98 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java
@@ -27,14 +27,15 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -58,7 +59,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -72,16 +73,16 @@
private final byte stringTypeTag = ATypeTag.SERIALIZED_STRING_TYPE_TAG;
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
evalList.evaluate(tuple, inputArgList);
byte[] serOrderedList = inputArgList.getByteArray();
int offset = inputArgList.getStartOffset();
- int size = 0;
+ int size;
if (ATypeTag.VALUE_TYPE_MAPPING[serOrderedList[offset]] != ATypeTag.ORDEREDLIST) {
- cannotProcessException(serOrderedList[offset], serOrderedList[offset + 1]);
+ throw new TypeMismatchException(getIdentifier().getName(), 0, serOrderedList[offset]);
} else {
switch (ATypeTag.VALUE_TYPE_MAPPING[serOrderedList[offset + 1]]) {
case INT8:
@@ -95,7 +96,7 @@
offset);
break;
default:
- cannotProcessException(serOrderedList[offset], serOrderedList[offset + 1]);
+ throw new UnsupportedTypeException(getIdentifier(), serOrderedList[offset]);
}
}
@@ -107,6 +108,7 @@
offset, i);
int codePoint = 0;
codePoint = ATypeHierarchy.getIntegerValueWithDifferentTypeTagPosition(
+ getIdentifier().getName(), 0,
serOrderedList, itemOffset, offset + 1);
utf_8_len += UTF8StringUtil.codePointToUTF8(codePoint, currentUTF8);
}
@@ -117,6 +119,7 @@
offset, i);
int codePoint = 0;
codePoint = ATypeHierarchy.getIntegerValueWithDifferentTypeTagPosition(
+ getIdentifier().getName(), 0,
serOrderedList, itemOffset, offset + 1);
utf_8_len = UTF8StringUtil.codePointToUTF8(codePoint, currentUTF8);
for (int j = 0; j < utf_8_len; j++) {
@@ -125,10 +128,10 @@
}
result.set(resultStorage);
} catch (AsterixException ex) {
- throw new AlgebricksException(ex);
+ throw new HyracksDataException(ex);
}
} catch (IOException e1) {
- throw new AlgebricksException(e1.getMessage());
+ throw new HyracksDataException(e1);
}
}
};
@@ -141,11 +144,4 @@
return AsterixBuiltinFunctions.CODEPOINT_TO_STRING;
}
- private void cannotProcessException(byte tag1, byte tag2) throws AlgebricksException {
- throw new AlgebricksException(AsterixBuiltinFunctions.CODEPOINT_TO_STRING.getName()
- + ": expects input type ORDEREDLIST/[INT8|INT16|INT32|INT64|FLOAT|DOUBLE] but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(tag1) + "/"
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(tag2));
- }
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java
index 7aa39bd..39e069d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java
@@ -29,6 +29,7 @@
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
@@ -55,7 +56,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
ITokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(3, true, false, true,
tokenFactory);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java
index 8a11e48..0dbf1e7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java
@@ -24,11 +24,11 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.WordTokensEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
@@ -50,13 +50,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
ITokenFactory tokenFactory = new HashedUTF8WordTokenFactory();
IBinaryTokenizer tokenizer = new DelimitedUTF8StringBinaryTokenizer(false, true, tokenFactory);
return new WordTokensEvaluator(args, ctx, tokenizer, BuiltinType.AINT32);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java
index fabbd26..989a246 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java
@@ -28,19 +28,19 @@
import org.apache.asterix.om.base.ACircle;
import org.apache.asterix.om.base.AMutableCircle;
import org.apache.asterix.om.base.AMutablePoint;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -58,13 +58,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -80,7 +79,7 @@
.getSerializerDeserializer(BuiltinType.ACIRCLE);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, inputArg0);
eval1.evaluate(tuple, inputArg1);
@@ -91,12 +90,13 @@
int offset1 = inputArg1.getStartOffset();
// Type check: (point, double)
- if (bytes0[offset0] != ATypeTag.SERIALIZED_POINT_TYPE_TAG
- || bytes1[offset1] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
- throw new AlgebricksException(AsterixBuiltinFunctions.CREATE_CIRCLE.getName()
- + ": expects input type (POINT, DOUBLE) but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]) + ")");
+ if (bytes0[offset0] != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG);
+ }
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
}
try {
@@ -108,7 +108,7 @@
aCircle.setValue(aPoint, ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1));
circleSerde.serialize(aCircle, out);
} catch (IOException e1) {
- throw new AlgebricksException(e1);
+ throw new HyracksDataException(e1);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateLineDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateLineDescriptor.java
index e86dbd7..2713388 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateLineDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateLineDescriptor.java
@@ -28,19 +28,19 @@
import org.apache.asterix.om.base.ALine;
import org.apache.asterix.om.base.AMutableLine;
import org.apache.asterix.om.base.AMutablePoint;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -58,13 +58,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -80,7 +79,7 @@
.getSerializerDeserializer(BuiltinType.ALINE);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, inputArg0);
eval1.evaluate(tuple, inputArg1);
@@ -90,12 +89,13 @@
int offset1 = inputArg1.getStartOffset();
// type-check: (point, point)
- if (bytes0[offset0] != ATypeTag.SERIALIZED_POINT_TYPE_TAG
- || bytes1[offset1] != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
- throw new AlgebricksException(AsterixBuiltinFunctions.CREATE_LINE.getName()
- + ": expects input type: (POINT, POINT) but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]) + ").");
+ if (bytes0[offset0] != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG);
+ }
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG);
}
try {
@@ -116,7 +116,7 @@
aLine.setValue(aPoint[0], aPoint[1]);
lineSerde.serialize(aLine, out);
} catch (IOException e1) {
- throw new AlgebricksException(e1);
+ throw new HyracksDataException(e1);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateMBRDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateMBRDescriptor.java
index a58f566..61ada60 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateMBRDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateMBRDescriptor.java
@@ -23,7 +23,6 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.CreateMBREvalFactory;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -37,7 +36,7 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new CreateMBREvalFactory(args[0], args[1], args[2]);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
index 6d37152..c01bba2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
@@ -25,19 +25,19 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.AMutablePoint;
import org.apache.asterix.om.base.APoint;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -53,13 +53,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -74,7 +73,7 @@
.getSerializerDeserializer(BuiltinType.APOINT);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval0.evaluate(tuple, inputArg0);
eval1.evaluate(tuple, inputArg1);
@@ -84,12 +83,13 @@
int offset1 = inputArg1.getStartOffset();
// type-check: (double, double)
- if (bytes0[offset0] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG
- || bytes1[offset1] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
- throw new AlgebricksException(AsterixBuiltinFunctions.CREATE_POINT.getName()
- + ": expects input type: (DOUBLE, DOUBLE) but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]) + ").");
+ if (bytes0[offset0] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+ }
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
}
resultStorage.reset();
@@ -98,7 +98,7 @@
ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1));
pointSerde.serialize(aPoint, out);
} catch (IOException e1) {
- throw new AlgebricksException(e1);
+ throw new HyracksDataException(e1);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java
index 9ba0e65..c87277b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java
@@ -25,21 +25,24 @@
import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.APointSerializerDeserializer;
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMissing;
import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.AsterixListAccessor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -62,7 +65,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final AsterixListAccessor listAccessor = new AsterixListAccessor();
@@ -74,25 +77,24 @@
@SuppressWarnings("unchecked")
private final ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.ANULL);
+ private final ISerializerDeserializer<AMissing> missingSerde = AqlSerializerDeserializerProvider.
+ INSTANCE.getSerializerDeserializer(BuiltinType.AMISSING);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
evalList.evaluate(tuple, inputArgList);
byte[] listBytes = inputArgList.getByteArray();
int offset = inputArgList.getStartOffset();
- if (listBytes[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
- throw new AlgebricksException(AsterixBuiltinFunctions.CREATE_POLYGON.getName()
- + ": expects input type ORDEREDLIST, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(listBytes[offset]));
+ if (listBytes[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
+ && listBytes[offset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, listBytes[offset],
+ ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
+ ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
}
- try {
- listAccessor.reset(listBytes, offset);
- } catch (AsterixException e) {
- throw new AlgebricksException(e);
- }
+ listAccessor.reset(listBytes, offset);
try {
// First check the list consists of a valid items
for (int i = 0; i < listAccessor.size(); i++) {
@@ -103,17 +105,21 @@
nullSerde.serialize(ANull.NULL, out);
return;
}
- throw new AlgebricksException(AsterixBuiltinFunctions.CREATE_POLYGON.getName()
- + ": expects type DOUBLE/NULL for the list item but got " + itemType);
+ if (itemType == ATypeTag.MISSING) {
+ missingSerde.serialize(AMissing.MISSING, out);
+ return;
+ }
+ throw new UnsupportedItemTypeException(AsterixBuiltinFunctions.CREATE_POLYGON,
+ itemType.serialize());
}
}
if (listAccessor.size() < 6) {
- throw new AlgebricksException(
- "A polygon instance must consists of at least 3 points");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
} else if (listAccessor.size() % 2 != 0) {
- throw new AlgebricksException(
- "There must be an even number of double values in the list to form a polygon");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
}
out.writeByte(ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
out.writeShort(listAccessor.size() / 2);
@@ -130,10 +136,10 @@
}
result.set(resultStorage);
} catch (AsterixException ex) {
- throw new AlgebricksException(ex);
+ throw new HyracksDataException(ex);
}
} catch (IOException e1) {
- throw new AlgebricksException(e1.getMessage());
+ throw new HyracksDataException(e1);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateQueryUIDDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateQueryUIDDescriptor.java
index a7131fb..8e49a82 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateQueryUIDDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateQueryUIDDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.IntegerPointable;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -47,14 +47,14 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
private static final int BINARY_LENGTH = 14;
private static final int PAYLOAD_START = 2;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
// Format: |TypeTag | PayloadLength | Payload |
// TypeTag: 1 byte
// PayloadLength: 1 byte
@@ -70,7 +70,7 @@
return new IScalarEvaluator() {
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
// Increments the Unique ID value.
for (int i = BINARY_LENGTH - 1; i >= PAYLOAD_START; i--) {
if (++uidBytes[i] != 0) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java
index 7087453..593e71d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java
@@ -28,19 +28,19 @@
import org.apache.asterix.om.base.AMutablePoint;
import org.apache.asterix.om.base.AMutableRectangle;
import org.apache.asterix.om.base.ARectangle;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -58,13 +58,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
@@ -80,7 +79,7 @@
.getSerializerDeserializer(BuiltinType.ARECTANGLE);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval0.evaluate(tuple, inputArg0);
eval1.evaluate(tuple, inputArg1);
@@ -90,12 +89,13 @@
int offset1 = inputArg1.getStartOffset();
resultStorage.reset();
- if (bytes0[offset0] != ATypeTag.SERIALIZED_POINT_TYPE_TAG
- || bytes1[offset1] != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
- throw new AlgebricksException(AsterixBuiltinFunctions.CREATE_RECTANGLE.getName()
- + ": expects input type: (POINT, POINT) but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]) + ".");
+ if (bytes0[offset0] != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG);
+ }
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG);
}
try {
@@ -124,7 +124,7 @@
rectangle2DSerde.serialize(aRectangle, out);
result.set(resultStorage);
} catch (IOException e1) {
- throw new AlgebricksException(e1);
+ throw new HyracksDataException(e1);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateUUIDDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateUUIDDescriptor.java
index cde092d..d1c40c3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateUUIDDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateUUIDDescriptor.java
@@ -28,7 +28,6 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -51,7 +50,7 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@@ -61,22 +60,18 @@
.getSerializerDeserializer(BuiltinType.AUUID);
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput output = resultStorage.getDataOutput();
private final AGeneratedUUID uuid = new AGeneratedUUID();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
uuid.nextUUID();
- try {
- resultStorage.reset();
- uuidSerDe.serialize(uuid, output);
- result.set(resultStorage);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ resultStorage.reset();
+ uuidSerDe.serialize(uuid, output);
+ result.set(resultStorage);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
index dec8829..dcbfcf1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/DeepEqualityDescriptor.java
@@ -32,12 +32,12 @@
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.comparisons.DeepEqualAssessor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -71,7 +71,7 @@
.getSerializerDeserializer(BuiltinType.ABOOLEAN);
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
final DataOutput out = resultStorage.getDataOutput();
final IScalarEvaluator evalLeft = evalFactoryLeft.createScalarEvaluator(ctx);
@@ -84,7 +84,7 @@
private final IVisitablePointable pointableRight = allocator.allocateFieldValue(inputTypeRight);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
evalLeft.evaluate(tuple, pointableLeft);
evalRight.evaluate(tuple, pointableRight);
@@ -97,7 +97,7 @@
boolSerde.serialize(resultBit, out);
result.set(resultStorage);
} catch (Exception ioe) {
- throw new AlgebricksException(ioe);
+ throw new HyracksDataException(ioe);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java
index f1cc899..360d12e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.EditDistanceCheckEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class EditDistanceCheckDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -40,13 +40,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new EditDistanceCheckEvaluator(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceContainsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceContainsDescriptor.java
index d9ef39e..296af09 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceContainsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceContainsDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.EditDistanceContainsEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class EditDistanceContainsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -40,13 +40,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new EditDistanceContainsEvaluator(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java
index 88b9b31..e0f9a90 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.EditDistanceEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class EditDistanceDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -40,13 +40,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new EditDistanceEvaluator(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceListIsFilterableDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceListIsFilterableDescriptor.java
index a4c4311..a7337a8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceListIsFilterableDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceListIsFilterableDescriptor.java
@@ -33,7 +33,7 @@
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -65,13 +65,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new EditDistanceListIsFilterableEvaluator(args, ctx);
}
};
@@ -97,13 +96,13 @@
.getSerializerDeserializer(BuiltinType.ABOOLEAN);
public EditDistanceListIsFilterableEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
listEval = args[0].createScalarEvaluator(context);
edThreshEval = args[1].createScalarEvaluator(context);
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
listEval.evaluate(tuple, listPtr);
@@ -123,21 +122,16 @@
listLen = AOrderedListSerializerDeserializer.getNumberOfItems(bytes, offset);
break;
default:
- throw new AlgebricksException(AsterixBuiltinFunctions.EDIT_DISTANCE_LIST_IS_FILTERABLE.getName()
- + ": expects input type ORDEREDLIST or UNORDEREDLIST as the first argument, but got "
- + typeTag + ".");
+ throw new TypeMismatchException(AsterixBuiltinFunctions.EDIT_DISTANCE_LIST_IS_FILTERABLE,
+ 0, ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG,
+ ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
}
// Check type and extract edit-distance threshold.
bytes = edThreshPtr.getByteArray();
offset = edThreshPtr.getStartOffset();
- long edThresh;
-
- try {
- edThresh = ATypeHierarchy.getIntegerValue(bytes, offset);
- } catch (HyracksDataException e1) {
- throw new AlgebricksException(e1);
- }
+ long edThresh = ATypeHierarchy.getIntegerValue(
+ AsterixBuiltinFunctions.EDIT_DISTANCE_LIST_IS_FILTERABLE.getName(), 1, bytes, offset);
// Compute result.
long lowerBound = listLen - edThresh;
@@ -148,7 +142,7 @@
booleanSerde.serialize(ABoolean.TRUE, output);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableDescriptor.java
index 8944453..cafb03b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableDescriptor.java
@@ -22,11 +22,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
/**
* Checks whether a string with an edit distance threshold can be filtered with a lower bounding
@@ -47,13 +47,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new EditDistanceStringIsFilterableEvaluator(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableEvaluator.java
index 595a85a..63584fa 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableEvaluator.java
@@ -27,9 +27,8 @@
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -63,7 +62,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
public EditDistanceStringIsFilterableEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
stringEval = args[0].createScalarEvaluator(context);
edThreshEval = args[1].createScalarEvaluator(context);
gramLenEval = args[2].createScalarEvaluator(context);
@@ -71,7 +70,7 @@
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
stringEval.evaluate(tuple, stringPtr);
@@ -80,37 +79,29 @@
usePrePostEval.evaluate(tuple, usePrePostPtr);
// Check type and compute string length.
- ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
- .deserialize(stringPtr.getByteArray()[stringPtr.getStartOffset()]);
- if (!typeTag.equals(ATypeTag.STRING)) {
- throw new AlgebricksException(AsterixBuiltinFunctions.EDIT_DISTANCE_STRING_IS_FILTERABLE.getName()
- + ": expects input type STRING as first argument, but got " + typeTag + ".");
+ byte typeTag = stringPtr.getByteArray()[stringPtr.getStartOffset()];
+ if (typeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(AsterixBuiltinFunctions.EDIT_DISTANCE_STRING_IS_FILTERABLE, 0, typeTag,
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
utf8Ptr.set(stringPtr.getByteArray(), stringPtr.getStartOffset() + 1, stringPtr.getLength());
int strLen = utf8Ptr.getStringLength();
// Check type and extract edit-distance threshold.
- long edThresh = 0;
- try {
- edThresh = ATypeHierarchy.getIntegerValue(edThreshPtr.getByteArray(), edThreshPtr.getStartOffset());
- } catch (HyracksDataException e1) {
- throw new AlgebricksException(e1);
- }
+ long edThresh = ATypeHierarchy.getIntegerValue(
+ AsterixBuiltinFunctions.EDIT_DISTANCE_LIST_IS_FILTERABLE.getName(), 1, edThreshPtr.getByteArray(),
+ edThreshPtr.getStartOffset());
// Check type and extract gram length.
- long gramLen = 0;
- try {
- gramLen = ATypeHierarchy.getIntegerValue(gramLenPtr.getByteArray(), gramLenPtr.getStartOffset());
- } catch (HyracksDataException e1) {
- throw new AlgebricksException(e1);
- }
+ long gramLen = ATypeHierarchy.getIntegerValue(
+ AsterixBuiltinFunctions.EDIT_DISTANCE_LIST_IS_FILTERABLE.getName(), 2, gramLenPtr.getByteArray(),
+ gramLenPtr.getStartOffset());
// Check type and extract usePrePost flag.
- typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
- .deserialize(usePrePostPtr.getByteArray()[usePrePostPtr.getStartOffset()]);
- if (!typeTag.equals(ATypeTag.BOOLEAN)) {
- throw new AlgebricksException(AsterixBuiltinFunctions.EDIT_DISTANCE_STRING_IS_FILTERABLE.getName()
- + ": expects input type BOOLEAN as fourth argument, but got " + typeTag + ".");
+ typeTag = usePrePostPtr.getByteArray()[usePrePostPtr.getStartOffset()];
+ if (typeTag != ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG) {
+ throw new TypeMismatchException(AsterixBuiltinFunctions.EDIT_DISTANCE_STRING_IS_FILTERABLE, 3, typeTag,
+ ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
}
boolean usePrePost = BooleanPointable.getBoolean(usePrePostPtr.getByteArray(),
usePrePostPtr.getStartOffset() + 1);
@@ -125,7 +116,7 @@
booleanSerde.serialize(ABoolean.TRUE, output);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
index 88b8a51..9f0da4e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
@@ -31,11 +31,12 @@
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.om.util.NonTaggedFormatUtil;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -76,7 +77,7 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -91,8 +92,7 @@
private int itemLength;
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
-
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
evalList.evaluate(tuple, inputArgList);
evalIdx.evaluate(tuple, inputArgIdx);
@@ -103,14 +103,11 @@
int indexOffset = inputArgIdx.getStartOffset();
if (serOrderedList[offset] == ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
- itemIndex = ATypeHierarchy.getIntegerValue(indexBytes, indexOffset);
+ itemIndex = ATypeHierarchy.getIntegerValue(AsterixBuiltinFunctions.GET_ITEM.getName(), 0,
+ indexBytes, indexOffset);
} else {
- throw new AlgebricksException(AsterixBuiltinFunctions.GET_ITEM.getName()
- + ": expects input type (NULL/ORDEREDLIST, [INT8/16/32/64/FLOAT/DOUBLE]), but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serOrderedList[offset]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER
- .deserialize(inputArgIdx.getByteArray()[offset])
- + ").");
+ throw new TypeMismatchException(AsterixBuiltinFunctions.GET_ITEM,
+ 0, serOrderedList[offset], ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
}
if (itemIndex < 0 || itemIndex >= AOrderedListSerializerDeserializer
@@ -144,9 +141,9 @@
result.set(resultStorage);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
} catch (AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GramTokensDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GramTokensDescriptor.java
index 3b7941e..f4ce63d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GramTokensDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GramTokensDescriptor.java
@@ -24,11 +24,11 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.GramTokensEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8NGramTokenFactory;
@@ -49,13 +49,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
ITokenFactory tokenFactory = new UTF8NGramTokenFactory();
NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(3, true, true, true,
tokenFactory);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java
index 575dacf..3c29ddb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java
@@ -24,11 +24,11 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.GramTokensEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
@@ -49,13 +49,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
ITokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(3, true, true, true,
tokenFactory);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java
index c32c6ae..25294ed 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java
@@ -24,11 +24,11 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.WordTokensEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
@@ -50,13 +50,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
ITokenFactory tokenFactory = new HashedUTF8WordTokenFactory();
IBinaryTokenizer tokenizer = new DelimitedUTF8StringBinaryTokenizer(true, true, tokenFactory);
return new WordTokensEvaluator(args, ctx, tokenizer, BuiltinType.AINT32);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java
index 0d1cf4d..d4bc02f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.runtime.evaluators.functions;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
@@ -25,11 +27,11 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -50,14 +52,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
-
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
final IPointable argPtr = new VoidPointable();
final IScalarEvaluator[] evals = new IScalarEvaluator[args.length];
@@ -67,7 +67,7 @@
return new IScalarEvaluator() {
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
evals[1].evaluate(tuple, argPtr);
ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
.deserialize(argPtr.getByteArray()[argPtr.getStartOffset()]);
@@ -75,8 +75,7 @@
boolean argResult = ABooleanSerializerDeserializer.getBoolean(argPtr.getByteArray(),
argPtr.getStartOffset() + 1);
if (argResult) {
- throw new AlgebricksException(
- AsterixBuiltinFunctions.INJECT_FAILURE + ": injecting a intended failure");
+ throw new RuntimeDataException(ErrorCode.ERROR_INJECTED_FAILURE, getIdentifier());
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsArrayDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsArrayDescriptor.java
index c8cb313..7d970d3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsArrayDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsArrayDescriptor.java
@@ -25,11 +25,11 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.AbstractTypeCheckEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class IsArrayDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
@@ -41,13 +41,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
final IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
return new AbstractTypeCheckEvaluator(eval) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsBooleanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsBooleanDescriptor.java
index 89318bd..6e03979 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsBooleanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsBooleanDescriptor.java
@@ -25,11 +25,11 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.AbstractTypeCheckEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class IsBooleanDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
@@ -41,13 +41,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
final IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
return new AbstractTypeCheckEvaluator(eval) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsMissingDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsMissingDescriptor.java
index a3bdb30..cdaf9a7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsMissingDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsMissingDescriptor.java
@@ -24,11 +24,11 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.AbstractTypeCheckEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class IsMissingDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
@@ -40,13 +40,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
final IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
return new AbstractTypeCheckEvaluator(eval) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
index 33db8f3..458bf0f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
@@ -24,11 +24,11 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.AbstractTypeCheckEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class IsNullDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
@@ -40,13 +40,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
final IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
return new AbstractTypeCheckEvaluator(eval) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNumberDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNumberDescriptor.java
index 418d778..f1d8b70 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNumberDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNumberDescriptor.java
@@ -25,11 +25,11 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.AbstractTypeCheckEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class IsNumberDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
@@ -41,13 +41,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
final IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
return new AbstractTypeCheckEvaluator(eval) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsObjectDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsObjectDescriptor.java
index 5a3f2d3..ea37914 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsObjectDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsObjectDescriptor.java
@@ -25,11 +25,11 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.AbstractTypeCheckEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class IsObjectDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
@@ -41,13 +41,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
final IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
return new AbstractTypeCheckEvaluator(eval) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsStringDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsStringDescriptor.java
index 44c5a6c..0bd11dc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsStringDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsStringDescriptor.java
@@ -25,11 +25,11 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.AbstractTypeCheckEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class IsStringDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
@@ -41,13 +41,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
final IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
return new AbstractTypeCheckEvaluator(eval) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java
index 7ff2c5f..abeda21 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -49,14 +48,13 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -66,16 +64,12 @@
private final AObjectSerializerDeserializer aObjSerDer = AObjectSerializerDeserializer.INSTANCE;
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval.evaluate(tuple, argPtr);
boolean isSystemNull = argPtr.getByteArray()[argPtr
.getStartOffset()] == ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG;
ABoolean res = isSystemNull ? ABoolean.TRUE : ABoolean.FALSE;
- try {
- aObjSerDer.serialize(res, out);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ aObjSerDer.serialize(res, out);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsUnknownDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsUnknownDescriptor.java
index ef5f42c..8d25ae2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsUnknownDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsUnknownDescriptor.java
@@ -24,11 +24,11 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.AbstractTypeCheckEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class IsUnknownDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
@@ -40,13 +40,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
final IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
return new AbstractTypeCheckEvaluator(eval) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java
index c681d78..e4c341c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java
@@ -26,19 +26,19 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -54,14 +54,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
-
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
@@ -77,7 +75,7 @@
.getSerializerDeserializer(BuiltinType.AINT64);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
evalList.evaluate(tuple, inputVal);
byte[] serList = inputVal.getByteArray();
@@ -85,9 +83,9 @@
if (serList[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
&& serList[offset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
- throw new AlgebricksException(AsterixBuiltinFunctions.LEN.getName()
- + ": expects input type ORDEREDLIST/UNORDEREDLIST but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serList[offset]));
+ throw new TypeMismatchException(getIdentifier(), 0,
+ serList[offset], ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
+ ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
}
int numberOfitems = 0;
@@ -101,7 +99,7 @@
try {
int64Serde.serialize(res, out);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java
index fb209a9..3771886 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java
@@ -29,7 +29,7 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -56,13 +56,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
@@ -71,30 +70,24 @@
private IPointable argPtr = new VoidPointable();
private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private String errorMessage = AsterixBuiltinFunctions.NOT.getName()
- + ": expects input type BOOLEAN/NULL";
@SuppressWarnings("unchecked")
private ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.ABOOLEAN);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int offset = argPtr.getStartOffset();
-
- try {
- if (bytes[offset] == ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG) {
- boolean argRes = ABooleanSerializerDeserializer.getBoolean(bytes, offset + 1);
- ABoolean aResult = argRes ? (ABoolean.FALSE) : (ABoolean.TRUE);
- booleanSerde.serialize(aResult, out);
- } else {
- throw new AlgebricksException(errorMessage);
- }
- } catch (HyracksDataException hde) {
- throw new AlgebricksException(hde);
+ if (bytes[offset] == ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG) {
+ boolean argRes = ABooleanSerializerDeserializer.getBoolean(bytes, offset + 1);
+ ABoolean aResult = argRes ? ABoolean.FALSE : ABoolean.TRUE;
+ booleanSerde.serialize(aResult, out);
+ } else {
+ throw new TypeMismatchException(getIdentifier(), 0,
+ bytes[offset], ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericACosDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericACosDescriptor.java
index 51609c3..dd7f916 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericACosDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericACosDescriptor.java
@@ -22,11 +22,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
public class NumericACosDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -49,7 +49,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new NumericACosEvaluator(ctx, args[0]);
}
};
@@ -58,12 +58,12 @@
private class NumericACosEvaluator extends AbstractUnaryNumericDoubleFunctionEval {
NumericACosEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
- throws AlgebricksException {
+ throws HyracksDataException {
super(context, argEvalFactory, NumericACosDescriptor.this.getIdentifier());
}
@Override
- protected void processDouble(double arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
aDouble.setValue(Math.acos(arg));
serialize(aDouble, doubleSerde, resultPointable);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericASinDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericASinDescriptor.java
index 1bd0f6e..8d5ab88 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericASinDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericASinDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
public class NumericASinDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -50,7 +50,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new NumericASinEvaluator(ctx, args[0]);
}
};
@@ -59,12 +59,12 @@
private class NumericASinEvaluator extends AbstractUnaryNumericDoubleFunctionEval {
NumericASinEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
- throws AlgebricksException {
+ throws HyracksDataException {
super(context, argEvalFactory, NumericASinDescriptor.this.getIdentifier());
}
@Override
- protected void processDouble(double arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
aDouble.setValue(Math.asin(arg));
serialize(aDouble, doubleSerde, resultPointable);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATan2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATan2Descriptor.java
index e7b51c0..f5e4144 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATan2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATan2Descriptor.java
@@ -28,7 +28,6 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -56,13 +55,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
// For inputs.
private final IPointable leftPtr = new VoidPointable();
@@ -81,23 +79,19 @@
@SuppressWarnings("unchecked")
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
- try {
- resultStorage.reset();
- evalLeft.evaluate(tuple, leftPtr);
- evalRight.evaluate(tuple, rightPtr);
- for (int i = 0; i < args.length; i++) {
- IPointable argPtr = i == 0 ? leftPtr : rightPtr;
- byte[] data = argPtr.getByteArray();
- int offset = argPtr.getStartOffset();
- operands[i] = ATypeHierarchy.getDoubleValue(data, offset);
- }
- aDouble.setValue(Math.atan2(operands[0], operands[1]));
- outputSerde.serialize(aDouble, out);
- result.set(resultStorage);
- } catch (HyracksDataException hde) {
- throw new AlgebricksException(hde);
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ evalLeft.evaluate(tuple, leftPtr);
+ evalRight.evaluate(tuple, rightPtr);
+ for (int i = 0; i < args.length; i++) {
+ IPointable argPtr = i == 0 ? leftPtr : rightPtr;
+ byte[] data = argPtr.getByteArray();
+ int offset = argPtr.getStartOffset();
+ operands[i] = ATypeHierarchy.getDoubleValue(getIdentifier().getName(), i, data, offset);
}
+ aDouble.setValue(Math.atan2(operands[0], operands[1]));
+ outputSerde.serialize(aDouble, out);
+ result.set(resultStorage);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATanDescriptor.java
index 2d5a359..6edda86 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATanDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
public class NumericATanDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -50,7 +50,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new NumericATanEvaluator(ctx, args[0]);
}
};
@@ -59,12 +59,12 @@
private class NumericATanEvaluator extends AbstractUnaryNumericDoubleFunctionEval {
NumericATanEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
- throws AlgebricksException {
+ throws HyracksDataException {
super(context, argEvalFactory, NumericATanDescriptor.this.getIdentifier());
}
@Override
- protected void processDouble(double arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
aDouble.setValue(Math.atan(arg));
serialize(aDouble, doubleSerde, resultPointable);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java
index 69edb6e..2ee75f5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java
@@ -27,11 +27,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
public class NumericAbsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -54,7 +54,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new NumericAbsEvaluator(ctx, args[0]);
}
};
@@ -63,12 +63,12 @@
private class NumericAbsEvaluator extends AbstractUnaryNumericFunctionEval {
NumericAbsEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
- throws AlgebricksException {
+ throws HyracksDataException {
super(context, argEvalFactory, NumericAbsDescriptor.this.getIdentifier());
}
@Override
- protected void processInt8(byte arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt8(byte arg, IPointable resultPointable) throws HyracksDataException {
if (arg >= 0) {
resultPointable.set(argPtr);
return;
@@ -78,7 +78,7 @@
}
@Override
- protected void processInt16(short arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt16(short arg, IPointable resultPointable) throws HyracksDataException {
if (arg >= 0) {
resultPointable.set(argPtr);
return;
@@ -88,7 +88,7 @@
}
@Override
- protected void processInt32(int arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt32(int arg, IPointable resultPointable) throws HyracksDataException {
if (arg >= 0) {
resultPointable.set(argPtr);
return;
@@ -98,7 +98,7 @@
}
@Override
- protected void processInt64(long arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt64(long arg, IPointable resultPointable) throws HyracksDataException {
if (arg >= 0) {
resultPointable.set(argPtr);
return;
@@ -108,7 +108,7 @@
}
@Override
- protected void processFloat(float arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processFloat(float arg, IPointable resultPointable) throws HyracksDataException {
if (arg >= 0.0f) {
resultPointable.set(argPtr);
return;
@@ -118,7 +118,7 @@
}
@Override
- protected void processDouble(double arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
if (arg >= 0.0d) {
resultPointable.set(argPtr);
return;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAddDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
index 265f68e..edbc21a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
@@ -22,9 +22,10 @@
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import java.lang.Math;
public class NumericAddDescriptor extends AbstractNumericArithmeticEval {
@@ -58,6 +59,6 @@
@Override
protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
- throw new HyracksDataException("Undefined addition operation between two time instances.");
+ throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
index 60f1233..ab1e032 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
@@ -18,14 +18,16 @@
*/
package org.apache.asterix.runtime.evaluators.functions;
-import com.google.common.math.LongMath;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import com.google.common.math.LongMath;
+
public class NumericCaretDescriptor extends AbstractNumericArithmeticEval {
private static final long serialVersionUID = 1L;
@@ -66,12 +68,12 @@
@Override
protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
throws HyracksDataException {
- throw new NotImplementedException("Caret operation is not defined for temporal types");
+ throw new UnsupportedTypeException(getIdentifier().getName(), ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
}
@Override
protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
- throw new NotImplementedException("Caret operation is not defined for temporal types");
+ throw new UnsupportedTypeException(getIdentifier().getName(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java
index 0ba9b00..ce7db1a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java
@@ -22,11 +22,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
public class NumericCeilingDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -50,7 +50,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new NumericCeilingEvaluator(ctx, args[0]);
}
};
@@ -59,38 +59,38 @@
private class NumericCeilingEvaluator extends AbstractUnaryNumericFunctionEval {
NumericCeilingEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
- throws AlgebricksException {
+ throws HyracksDataException {
super(context, argEvalFactory, NumericCeilingDescriptor.this.getIdentifier());
}
@Override
- protected void processInt8(byte arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt8(byte arg, IPointable resultPointable) throws HyracksDataException {
resultPointable.set(argPtr);
}
@Override
- protected void processInt16(short arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt16(short arg, IPointable resultPointable) throws HyracksDataException {
resultPointable.set(argPtr);
}
@Override
- protected void processInt32(int arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt32(int arg, IPointable resultPointable) throws HyracksDataException {
resultPointable.set(argPtr);
}
@Override
- protected void processInt64(long arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt64(long arg, IPointable resultPointable) throws HyracksDataException {
resultPointable.set(argPtr);
}
@Override
- protected void processFloat(float arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processFloat(float arg, IPointable resultPointable) throws HyracksDataException {
aFloat.setValue((float) Math.ceil(arg));
serialize(aFloat, floatSerde, resultPointable);
}
@Override
- protected void processDouble(double arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
aDouble.setValue(Math.ceil(arg));
serialize(aDouble, doubleSerde, resultPointable);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCosDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCosDescriptor.java
index aaecd0a..f165b9c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCosDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCosDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
public class NumericCosDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -50,7 +50,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new NumericCosEvaluator(ctx, args[0]);
}
};
@@ -59,12 +59,12 @@
private class NumericCosEvaluator extends AbstractUnaryNumericDoubleFunctionEval {
NumericCosEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
- throws AlgebricksException {
+ throws HyracksDataException {
super(context, argEvalFactory, NumericCosDescriptor.this.getIdentifier());
}
@Override
- protected void processDouble(double arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
aDouble.setValue(Math.cos(arg));
serialize(aDouble, doubleSerde, resultPointable);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
index 1c5cb9c..80dd990 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
@@ -21,6 +21,8 @@
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -63,6 +65,6 @@
@Override
protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
- throw new NotImplementedException("Divide operation is not defined for temporal types");
+ throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericExpDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericExpDescriptor.java
index efdd85a..7840aba 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericExpDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericExpDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
public class NumericExpDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -50,7 +50,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new NumericExpEvaluator(ctx, args[0]);
}
};
@@ -59,12 +59,12 @@
private class NumericExpEvaluator extends AbstractUnaryNumericDoubleFunctionEval {
NumericExpEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
- throws AlgebricksException {
+ throws HyracksDataException {
super(context, argEvalFactory, NumericExpDescriptor.this.getIdentifier());
}
@Override
- protected void processDouble(double arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
aDouble.setValue(Math.exp(arg));
serialize(aDouble, doubleSerde, resultPointable);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java
index 2a04250..21963c2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java
@@ -27,11 +27,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
public class NumericFloorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -54,7 +54,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new NumericFloorEvaluator(ctx, args[0]);
}
};
@@ -63,38 +63,38 @@
private class NumericFloorEvaluator extends AbstractUnaryNumericFunctionEval {
NumericFloorEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
- throws AlgebricksException {
+ throws HyracksDataException {
super(context, argEvalFactory, NumericFloorDescriptor.this.getIdentifier());
}
@Override
- protected void processInt8(byte arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt8(byte arg, IPointable resultPointable) throws HyracksDataException {
resultPointable.set(argPtr);
}
@Override
- protected void processInt16(short arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt16(short arg, IPointable resultPointable) throws HyracksDataException {
resultPointable.set(argPtr);
}
@Override
- protected void processInt32(int arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt32(int arg, IPointable resultPointable) throws HyracksDataException {
resultPointable.set(argPtr);
}
@Override
- protected void processInt64(long arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt64(long arg, IPointable resultPointable) throws HyracksDataException {
resultPointable.set(argPtr);
}
@Override
- protected void processFloat(float arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processFloat(float arg, IPointable resultPointable) throws HyracksDataException {
aFloat.setValue((float) Math.floor(arg));
serialize(aFloat, floatSerde, resultPointable);
}
@Override
- protected void processDouble(double arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
aDouble.setValue(Math.floor(arg));
serialize(aDouble, doubleSerde, resultPointable);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLnDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLnDescriptor.java
index d8e8c5c..59735b9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLnDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLnDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
public class NumericLnDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -50,7 +50,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new NumericLnEvaluator(ctx, args[0]);
}
};
@@ -59,12 +59,12 @@
private class NumericLnEvaluator extends AbstractUnaryNumericDoubleFunctionEval {
NumericLnEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
- throws AlgebricksException {
+ throws HyracksDataException {
super(context, argEvalFactory, NumericLnDescriptor.this.getIdentifier());
}
@Override
- protected void processDouble(double arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
aDouble.setValue(Math.log(arg));
serialize(aDouble, doubleSerde, resultPointable);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLogDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLogDescriptor.java
index 336bfc4..7e49d88 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLogDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLogDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
public class NumericLogDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -50,7 +50,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new NumericLogEvaluator(ctx, args[0]);
}
};
@@ -59,12 +59,12 @@
private class NumericLogEvaluator extends AbstractUnaryNumericDoubleFunctionEval {
NumericLogEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
- throws AlgebricksException {
+ throws HyracksDataException {
super(context, argEvalFactory, NumericLogDescriptor.this.getIdentifier());
}
@Override
- protected void processDouble(double arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
aDouble.setValue(Math.log10(arg));
serialize(aDouble, doubleSerde, resultPointable);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java
index 335dc41..2a66aa7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java
@@ -21,7 +21,8 @@
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -54,11 +55,11 @@
@Override
protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
throws HyracksDataException {
- throw new NotImplementedException("Multiply operation is not defined for temporal types");
+ throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
}
@Override
protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
- throw new NotImplementedException("Multiply operation is not defined for temporal types");
+ throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java
index c3f6580..a71aa96 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java
@@ -21,10 +21,10 @@
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import java.lang.Math;
public class NumericMultiplyDescriptor extends AbstractNumericArithmeticEval {
@@ -53,11 +53,11 @@
@Override
protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
throws HyracksDataException {
- throw new NotImplementedException("Multiply operation is not defined for temporal types");
+ throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
}
@Override
protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
- throw new NotImplementedException("Multiply operation is not defined for temporal types");
+ throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundDescriptor.java
index 773ba27..58e916c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundDescriptor.java
@@ -38,15 +38,13 @@
import org.apache.asterix.om.base.AMutableInt32;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.AMutableInt8;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -78,7 +76,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
@@ -97,58 +95,55 @@
@SuppressWarnings("unchecked")
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval.evaluate(tuple, argPtr);
byte[] data = argPtr.getByteArray();
int offset = argPtr.getStartOffset();
- try {
- if (data[offset] == ATypeTag.SERIALIZED_INT8_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT8);
- byte val = AInt8SerializerDeserializer.getByte(data, offset + 1);
- aInt8.setValue(val);
- serde.serialize(aInt8, out);
- } else if (data[offset] == ATypeTag.SERIALIZED_INT16_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT16);
- short val = AInt16SerializerDeserializer.getShort(data, offset + 1);
- aInt16.setValue(val);
- serde.serialize(aInt16, out);
- } else if (data[offset] == ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT32);
- int val = AInt32SerializerDeserializer.getInt(data, offset + 1);
- aInt32.setValue(val);
- serde.serialize(aInt32, out);
- } else if (data[offset] == ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT64);
- long val = AInt64SerializerDeserializer.getLong(data, offset + 1);
- aInt64.setValue(val);
- serde.serialize(aInt64, out);
- } else if (data[offset] == ATypeTag.SERIALIZED_FLOAT_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AFLOAT);
- float val = AFloatSerializerDeserializer.getFloat(data, offset + 1);
- val = Math.round(val);
- aFloat.setValue(val);
- serde.serialize(aFloat, out);
- } else if (data[offset] == ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADOUBLE);
- double val = ADoubleSerializerDeserializer.getDouble(data, offset + 1);
- val = Math.round(val);
- aDouble.setValue(val);
- serde.serialize(aDouble, out);
- } else {
- throw new NotImplementedException(
- AsterixBuiltinFunctions.NUMERIC_ROUND.getName() + ": not implemented for "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[offset]));
- }
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
+ if (data[offset] == ATypeTag.SERIALIZED_INT8_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT8);
+ byte val = AInt8SerializerDeserializer.getByte(data, offset + 1);
+ aInt8.setValue(val);
+ serde.serialize(aInt8, out);
+ } else if (data[offset] == ATypeTag.SERIALIZED_INT16_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT16);
+ short val = AInt16SerializerDeserializer.getShort(data, offset + 1);
+ aInt16.setValue(val);
+ serde.serialize(aInt16, out);
+ } else if (data[offset] == ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT32);
+ int val = AInt32SerializerDeserializer.getInt(data, offset + 1);
+ aInt32.setValue(val);
+ serde.serialize(aInt32, out);
+ } else if (data[offset] == ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT64);
+ long val = AInt64SerializerDeserializer.getLong(data, offset + 1);
+ aInt64.setValue(val);
+ serde.serialize(aInt64, out);
+ } else if (data[offset] == ATypeTag.SERIALIZED_FLOAT_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AFLOAT);
+ float val = AFloatSerializerDeserializer.getFloat(data, offset + 1);
+ val = Math.round(val);
+ aFloat.setValue(val);
+ serde.serialize(aFloat, out);
+ } else if (data[offset] == ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADOUBLE);
+ double val = ADoubleSerializerDeserializer.getDouble(data, offset + 1);
+ val = Math.round(val);
+ aDouble.setValue(val);
+ serde.serialize(aDouble, out);
+ } else {
+ throw new TypeMismatchException(getIdentifier(), 0, data[offset],
+ ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
+ ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEven2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEven2Descriptor.java
index e8d97a7..59b3355 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEven2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEven2Descriptor.java
@@ -39,15 +39,13 @@
import org.apache.asterix.om.base.AMutableInt32;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.AMutableInt8;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -80,7 +78,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
@@ -99,7 +97,7 @@
@SuppressWarnings("rawtypes")
private ISerializerDeserializer serde;
- private int getPrecision() throws AlgebricksException {
+ private int getPrecision() throws HyracksDataException {
byte[] bytes = argPrecision.getByteArray();
int offset = argPrecision.getStartOffset();
@@ -112,81 +110,76 @@
} else if (bytes[offset] == ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
return (int) AInt64SerializerDeserializer.getLong(bytes, offset + 1);
} else {
- throw new AlgebricksException(AsterixBuiltinFunctions.NUMERIC_ROUND_HALF_TO_EVEN2.getName()
- + ": the precision argument should be an INT8/INT16/INT32/INT64.");
+ throw new TypeMismatchException(getIdentifier(), 1, bytes[offset],
+ ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
}
}
@SuppressWarnings("unchecked")
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval.evaluate(tuple, argValue);
precision.evaluate(tuple, argPrecision);
byte[] data = argValue.getByteArray();
int offset = argValue.getStartOffset();
- try {
- if (data[offset] == ATypeTag.SERIALIZED_INT8_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT8);
- byte val = AInt8SerializerDeserializer.getByte(data, offset + 1);
- aInt8.setValue(val);
- serde.serialize(aInt8, out);
- } else if (data[offset] == ATypeTag.SERIALIZED_INT16_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT16);
- short val = AInt16SerializerDeserializer.getShort(data, offset + 1);
- aInt16.setValue(val);
- serde.serialize(aInt16, out);
- } else if (data[offset] == ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT32);
- int val = AInt32SerializerDeserializer.getInt(data, offset + 1);
- aInt32.setValue(val);
- serde.serialize(aInt32, out);
- } else if (data[offset] == ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT64);
- long val = AInt64SerializerDeserializer.getLong(data, offset + 1);
- aInt64.setValue(val);
- serde.serialize(aInt64, out);
- } else if (data[offset] == ATypeTag.SERIALIZED_FLOAT_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AFLOAT);
- float val = AFloatSerializerDeserializer.getFloat(data, offset + 1);
- if (Float.isNaN(val) || Float.isInfinite(val) || Float.compare(val, -0.0F) == 0
- || Float.compare(val, 0.0F) == 0) {
- aFloat.setValue(val);
- serde.serialize(aFloat, out);
- } else {
- BigDecimal r = new BigDecimal(Float.toString(val));
- aFloat.setValue(
- r.setScale(getPrecision(), BigDecimal.ROUND_HALF_EVEN).floatValue());
- serde.serialize(aFloat, out);
- }
- } else if (data[offset] == ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADOUBLE);
- double val = ADoubleSerializerDeserializer.getDouble(data, offset + 1);
- if (Double.isNaN(val) || Double.isInfinite(val) || Double.compare(val, -0.0D) == 0
- || Double.compare(val, 0.0D) == 0) {
- aDouble.setValue(val);
- serde.serialize(aDouble, out);
- } else {
- BigDecimal r = new BigDecimal(Double.toString(val));
- aDouble.setValue(
- r.setScale(getPrecision(), BigDecimal.ROUND_HALF_EVEN).doubleValue());
- serde.serialize(aDouble, out);
- }
+ if (data[offset] == ATypeTag.SERIALIZED_INT8_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT8);
+ byte val = AInt8SerializerDeserializer.getByte(data, offset + 1);
+ aInt8.setValue(val);
+ serde.serialize(aInt8, out);
+ } else if (data[offset] == ATypeTag.SERIALIZED_INT16_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT16);
+ short val = AInt16SerializerDeserializer.getShort(data, offset + 1);
+ aInt16.setValue(val);
+ serde.serialize(aInt16, out);
+ } else if (data[offset] == ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT32);
+ int val = AInt32SerializerDeserializer.getInt(data, offset + 1);
+ aInt32.setValue(val);
+ serde.serialize(aInt32, out);
+ } else if (data[offset] == ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT64);
+ long val = AInt64SerializerDeserializer.getLong(data, offset + 1);
+ aInt64.setValue(val);
+ serde.serialize(aInt64, out);
+ } else if (data[offset] == ATypeTag.SERIALIZED_FLOAT_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AFLOAT);
+ float val = AFloatSerializerDeserializer.getFloat(data, offset + 1);
+ if (Float.isNaN(val) || Float.isInfinite(val) || Float.compare(val, -0.0F) == 0
+ || Float.compare(val, 0.0F) == 0) {
+ aFloat.setValue(val);
+ serde.serialize(aFloat, out);
} else {
- throw new NotImplementedException(
- AsterixBuiltinFunctions.NUMERIC_ROUND_HALF_TO_EVEN2.getName()
- + ": not implemented for "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[offset]));
+ BigDecimal r = new BigDecimal(Float.toString(val));
+ aFloat.setValue(r.setScale(getPrecision(), BigDecimal.ROUND_HALF_EVEN).floatValue());
+ serde.serialize(aFloat, out);
}
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
+ } else if (data[offset] == ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADOUBLE);
+ double val = ADoubleSerializerDeserializer.getDouble(data, offset + 1);
+ if (Double.isNaN(val) || Double.isInfinite(val) || Double.compare(val, -0.0D) == 0
+ || Double.compare(val, 0.0D) == 0) {
+ aDouble.setValue(val);
+ serde.serialize(aDouble, out);
+ } else {
+ BigDecimal r = new BigDecimal(Double.toString(val));
+ aDouble.setValue(r.setScale(getPrecision(), BigDecimal.ROUND_HALF_EVEN).doubleValue());
+ serde.serialize(aDouble, out);
+ }
+ } else {
+ throw new TypeMismatchException(getIdentifier(), 0, data[offset],
+ ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
+ ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java
index 575035e..522c6cb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java
@@ -38,15 +38,13 @@
import org.apache.asterix.om.base.AMutableInt32;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.AMutableInt8;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -78,7 +76,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
@@ -97,59 +95,55 @@
@SuppressWarnings("unchecked")
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval.evaluate(tuple, argPtr);
byte[] data = argPtr.getByteArray();
int offset = argPtr.getStartOffset();
- try {
- if (data[offset] == ATypeTag.SERIALIZED_INT8_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT8);
- byte val = AInt8SerializerDeserializer.getByte(data, offset + 1);
- aInt8.setValue(val);
- serde.serialize(aInt8, out);
- } else if (data[offset] == ATypeTag.SERIALIZED_INT16_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT16);
- short val = AInt16SerializerDeserializer.getShort(data, offset + 1);
- aInt16.setValue(val);
- serde.serialize(aInt16, out);
- } else if (data[offset] == ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT32);
- int val = AInt32SerializerDeserializer.getInt(data, offset + 1);
- aInt32.setValue(val);
- serde.serialize(aInt32, out);
- } else if (data[offset] == ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT64);
- long val = AInt64SerializerDeserializer.getLong(data, offset + 1);
- aInt64.setValue(val);
- serde.serialize(aInt64, out);
- } else if (data[offset] == ATypeTag.SERIALIZED_FLOAT_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AFLOAT);
- float val = AFloatSerializerDeserializer.getFloat(data, offset + 1);
- aFloat.setValue((float) Math.rint(val));
- serde.serialize(aFloat, out);
- } else if (data[offset] == ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADOUBLE);
- double val = ADoubleSerializerDeserializer.getDouble(data, offset + 1);
- aDouble.setValue(Math.rint(val));
- serde.serialize(aDouble, out);
- } else {
- throw new NotImplementedException(
- AsterixBuiltinFunctions.NUMERIC_ROUND_HALF_TO_EVEN.getName()
- + ": not implemented for "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[offset]));
- }
- result.set(resultStorage);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
+ if (data[offset] == ATypeTag.SERIALIZED_INT8_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT8);
+ byte val = AInt8SerializerDeserializer.getByte(data, offset + 1);
+ aInt8.setValue(val);
+ serde.serialize(aInt8, out);
+ } else if (data[offset] == ATypeTag.SERIALIZED_INT16_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT16);
+ short val = AInt16SerializerDeserializer.getShort(data, offset + 1);
+ aInt16.setValue(val);
+ serde.serialize(aInt16, out);
+ } else if (data[offset] == ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT32);
+ int val = AInt32SerializerDeserializer.getInt(data, offset + 1);
+ aInt32.setValue(val);
+ serde.serialize(aInt32, out);
+ } else if (data[offset] == ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT64);
+ long val = AInt64SerializerDeserializer.getLong(data, offset + 1);
+ aInt64.setValue(val);
+ serde.serialize(aInt64, out);
+ } else if (data[offset] == ATypeTag.SERIALIZED_FLOAT_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AFLOAT);
+ float val = AFloatSerializerDeserializer.getFloat(data, offset + 1);
+ aFloat.setValue((float) Math.rint(val));
+ serde.serialize(aFloat, out);
+ } else if (data[offset] == ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADOUBLE);
+ double val = ADoubleSerializerDeserializer.getDouble(data, offset + 1);
+ aDouble.setValue(Math.rint(val));
+ serde.serialize(aDouble, out);
+ } else {
+ throw new TypeMismatchException(getIdentifier(), 0, data[offset],
+ ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
+ ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
}
+ result.set(resultStorage);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSignDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSignDescriptor.java
index 8fda156..5d9b916 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSignDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSignDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
public class NumericSignDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -50,7 +50,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new NumericSignEvaluator(ctx, args[0]);
}
};
@@ -59,42 +59,42 @@
private class NumericSignEvaluator extends AbstractUnaryNumericFunctionEval {
NumericSignEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
- throws AlgebricksException {
+ throws HyracksDataException {
super(context, argEvalFactory, NumericSignDescriptor.this.getIdentifier());
}
@Override
- protected void processInt8(byte arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt8(byte arg, IPointable resultPointable) throws HyracksDataException {
aInt8.setValue(arg >= 0 ? (byte) (arg > 0 ? 1 : 0) : -1);
serialize(aInt8, int8Serde, resultPointable);
}
@Override
- protected void processInt16(short arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt16(short arg, IPointable resultPointable) throws HyracksDataException {
aInt8.setValue(arg >= 0 ? (byte) (arg > 0 ? 1 : 0) : -1);
serialize(aInt8, int8Serde, resultPointable);
}
@Override
- protected void processInt32(int arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt32(int arg, IPointable resultPointable) throws HyracksDataException {
aInt8.setValue(arg >= 0 ? (byte) (arg > 0 ? 1 : 0) : -1);
serialize(aInt8, int8Serde, resultPointable);
}
@Override
- protected void processInt64(long arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt64(long arg, IPointable resultPointable) throws HyracksDataException {
aInt8.setValue(arg >= 0L ? (byte) (arg > 0L ? 1 : 0) : -1);
serialize(aInt8, int8Serde, resultPointable);
}
@Override
- protected void processFloat(float arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processFloat(float arg, IPointable resultPointable) throws HyracksDataException {
aInt8.setValue(arg >= 0.0f ? (byte) (arg > 0.0f ? 1 : 0) : -1);
serialize(aInt8, int8Serde, resultPointable);
}
@Override
- protected void processDouble(double arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
aInt8.setValue(arg >= 0.0d ? (byte) (arg > 0.0d ? 1 : 0) : -1);
serialize(aInt8, int8Serde, resultPointable);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinDescriptor.java
index 546e1a1..b709009 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
public class NumericSinDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -50,7 +50,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new NumericSinEvaluator(ctx, args[0]);
}
};
@@ -59,12 +59,12 @@
private class NumericSinEvaluator extends AbstractUnaryNumericDoubleFunctionEval {
NumericSinEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
- throws AlgebricksException {
+ throws HyracksDataException {
super(context, argEvalFactory, NumericSinDescriptor.this.getIdentifier());
}
@Override
- protected void processDouble(double arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
aDouble.setValue(Math.sin(arg));
serialize(aDouble, doubleSerde, resultPointable);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSqrtDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSqrtDescriptor.java
index aad8551..698b743 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSqrtDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSqrtDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
public class NumericSqrtDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -50,7 +50,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new NumericSqrtEvaluator(ctx, args[0]);
}
};
@@ -59,12 +59,12 @@
private class NumericSqrtEvaluator extends AbstractUnaryNumericDoubleFunctionEval {
NumericSqrtEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
- throws AlgebricksException {
+ throws HyracksDataException {
super(context, argEvalFactory, NumericSqrtDescriptor.this.getIdentifier());
}
@Override
- protected void processDouble(double arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
aDouble.setValue(Math.sqrt(arg));
serialize(aDouble, doubleSerde, resultPointable);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java
index 4355be0..9d02b8d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java
@@ -40,8 +40,7 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -69,13 +68,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -99,92 +97,82 @@
@SuppressWarnings("unchecked")
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
- try {
- resultStorage.reset();
- for (int i = 0; i < args.length; i++) {
- if (i == 0) {
- evalLeft.evaluate(tuple, argPtr);
- } else {
- evalRight.evaluate(tuple, argPtr);
- }
- byte[] data = argPtr.getByteArray();
- int offset = argPtr.getStartOffset();
- typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[offset]);
- switch (typeTag) {
- case INT8: {
- metInt8 = true;
- operands[i] = AInt8SerializerDeserializer.getByte(data, offset + 1);
- break;
- }
- case INT16: {
- metInt16 = true;
- operands[i] = AInt16SerializerDeserializer.getShort(data, offset + 1);
- break;
- }
- case INT32: {
- metInt32 = true;
- operands[i] = AInt32SerializerDeserializer.getInt(data, offset + 1);
- break;
- }
- case INT64: {
- metInt64 = true;
- operands[i] = AInt64SerializerDeserializer.getLong(data, offset + 1);
- break;
- }
- case FLOAT: {
- metFloat = true;
- operands[i] = AFloatSerializerDeserializer.getFloat(data, offset + 1);
- break;
- }
- case DOUBLE: {
- metDouble = true;
- operands[i] = ADoubleSerializerDeserializer.getDouble(data, offset + 1);
- break;
- }
- default: {
- throw new NotImplementedException(AsterixBuiltinFunctions.NUMERIC_SUBTRACT
- .getName() + (i == 0 ? ": left" : ": right") + " operand can not be "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[offset]));
- }
- }
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ for (int i = 0; i < args.length; i++) {
+ if (i == 0) {
+ evalLeft.evaluate(tuple, argPtr);
+ } else {
+ evalRight.evaluate(tuple, argPtr);
}
-
- if (metDouble) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADOUBLE);
- aDouble.setValue(operands[0] - operands[1]);
- serde.serialize(aDouble, out);
- } else if (metFloat) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AFLOAT);
- aFloat.setValue((float) (operands[0] - operands[1]));
- serde.serialize(aFloat, out);
- } else if (metInt64) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT64);
- aInt64.setValue((long) (operands[0] - operands[1]));
- serde.serialize(aInt64, out);
- } else if (metInt32) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT32);
- aInt32.setValue((int) (operands[0] - operands[1]));
- serde.serialize(aInt32, out);
- } else if (metInt16) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT16);
- aInt16.setValue((short) (operands[0] - operands[1]));
- serde.serialize(aInt16, out);
- } else if (metInt8) {
- serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT8);
- aInt8.setValue((byte) (operands[0] - operands[1]));
- serde.serialize(aInt8, out);
+ byte[] data = argPtr.getByteArray();
+ int offset = argPtr.getStartOffset();
+ typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[offset]);
+ switch (typeTag) {
+ case INT8:
+ metInt8 = true;
+ operands[i] = AInt8SerializerDeserializer.getByte(data, offset + 1);
+ break;
+ case INT16:
+ metInt16 = true;
+ operands[i] = AInt16SerializerDeserializer.getShort(data, offset + 1);
+ break;
+ case INT32:
+ metInt32 = true;
+ operands[i] = AInt32SerializerDeserializer.getInt(data, offset + 1);
+ break;
+ case INT64:
+ metInt64 = true;
+ operands[i] = AInt64SerializerDeserializer.getLong(data, offset + 1);
+ break;
+ case FLOAT:
+ metFloat = true;
+ operands[i] = AFloatSerializerDeserializer.getFloat(data, offset + 1);
+ break;
+ case DOUBLE:
+ metDouble = true;
+ operands[i] = ADoubleSerializerDeserializer.getDouble(data, offset + 1);
+ break;
+ default:
+ throw new TypeMismatchException(getIdentifier(), i, data[offset],
+ ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
+ ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
}
- result.set(resultStorage);
- } catch (HyracksDataException hde) {
- throw new AlgebricksException(hde);
}
+
+ if (metDouble) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADOUBLE);
+ aDouble.setValue(operands[0] - operands[1]);
+ serde.serialize(aDouble, out);
+ } else if (metFloat) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AFLOAT);
+ aFloat.setValue((float) (operands[0] - operands[1]));
+ serde.serialize(aFloat, out);
+ } else if (metInt64) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT64);
+ aInt64.setValue((long) (operands[0] - operands[1]));
+ serde.serialize(aInt64, out);
+ } else if (metInt32) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT32);
+ aInt32.setValue((int) (operands[0] - operands[1]));
+ serde.serialize(aInt32, out);
+ } else if (metInt16) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT16);
+ aInt16.setValue((short) (operands[0] - operands[1]));
+ serde.serialize(aInt16, out);
+ } else if (metInt8) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT8);
+ aInt8.setValue((byte) (operands[0] - operands[1]));
+ serde.serialize(aInt8, out);
+ }
+ result.set(resultStorage);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanDescriptor.java
index 5986e9c..f2a7878 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
public class NumericTanDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -50,7 +50,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new NumericTanEvaluator(ctx, args[0]);
}
};
@@ -59,12 +59,12 @@
private class NumericTanEvaluator extends AbstractUnaryNumericDoubleFunctionEval {
NumericTanEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
- throws AlgebricksException {
+ throws HyracksDataException {
super(context, argEvalFactory, NumericTanDescriptor.this.getIdentifier());
}
@Override
- protected void processDouble(double arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
aDouble.setValue(Math.tan(arg));
serialize(aDouble, doubleSerde, resultPointable);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTruncDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTruncDescriptor.java
index 6900027..cd7251e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTruncDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTruncDescriptor.java
@@ -39,15 +39,13 @@
import org.apache.asterix.om.base.AMutableInt32;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.AMutableInt8;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -80,7 +78,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new NumericTruncEvaluator(args, ctx);
}
};
@@ -102,12 +100,12 @@
@SuppressWarnings("rawtypes")
private ISerializerDeserializer serde;
- NumericTruncEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws AlgebricksException {
+ NumericTruncEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
eval = args[0].createScalarEvaluator(ctx);
precision = args[1].createScalarEvaluator(ctx);
}
- private int getPrecision() throws AlgebricksException {
+ private int getPrecision() throws HyracksDataException {
byte[] bytes = argPrecision.getByteArray();
int offset = argPrecision.getStartOffset();
@@ -120,72 +118,70 @@
} else if (bytes[offset] == ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
return (int) AInt64SerializerDeserializer.getLong(bytes, offset + 1);
} else {
- throw new AlgebricksException(NumericTruncDescriptor.this.getIdentifier().getName()
- + ": the precision argument should be an INT8/INT16/INT32/INT64.");
+ throw new TypeMismatchException(getIdentifier(), 1, bytes[offset],
+ ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
}
}
@SuppressWarnings("unchecked")
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval.evaluate(tuple, argValue);
precision.evaluate(tuple, argPrecision);
byte[] data = argValue.getByteArray();
int offset = argValue.getStartOffset();
- try {
- if (data[offset] == ATypeTag.SERIALIZED_INT8_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT8);
- byte val = AInt8SerializerDeserializer.getByte(data, offset + 1);
- aInt8.setValue(val);
- serde.serialize(aInt8, out);
- } else if (data[offset] == ATypeTag.SERIALIZED_INT16_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT16);
- short val = AInt16SerializerDeserializer.getShort(data, offset + 1);
- aInt16.setValue(val);
- serde.serialize(aInt16, out);
- } else if (data[offset] == ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
- int val = AInt32SerializerDeserializer.getInt(data, offset + 1);
- aInt32.setValue(val);
- serde.serialize(aInt32, out);
- } else if (data[offset] == ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
- long val = AInt64SerializerDeserializer.getLong(data, offset + 1);
- aInt64.setValue(val);
- serde.serialize(aInt64, out);
- } else if (data[offset] == ATypeTag.SERIALIZED_FLOAT_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AFLOAT);
- float val = AFloatSerializerDeserializer.getFloat(data, offset + 1);
- if (Float.isNaN(val) || Float.isInfinite(val) || Float.compare(val, -0.0F) == 0
- || Float.compare(val, 0.0F) == 0) {
- aFloat.setValue(val);
- serde.serialize(aFloat, out);
- } else {
- BigDecimal r = new BigDecimal(Float.toString(val));
- aFloat.setValue(r.setScale(getPrecision(), BigDecimal.ROUND_DOWN).floatValue());
- serde.serialize(aFloat, out);
- }
- } else if (data[offset] == ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
- serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
- double val = ADoubleSerializerDeserializer.getDouble(data, offset + 1);
- if (Double.isNaN(val) || Double.isInfinite(val) || Double.compare(val, -0.0D) == 0
- || Double.compare(val, 0.0D) == 0) {
- aDouble.setValue(val);
- serde.serialize(aDouble, out);
- } else {
- BigDecimal r = new BigDecimal(Double.toString(val));
- aDouble.setValue(r.setScale(getPrecision(), BigDecimal.ROUND_DOWN).doubleValue());
- serde.serialize(aDouble, out);
- }
+ if (data[offset] == ATypeTag.SERIALIZED_INT8_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT8);
+ byte val = AInt8SerializerDeserializer.getByte(data, offset + 1);
+ aInt8.setValue(val);
+ serde.serialize(aInt8, out);
+ } else if (data[offset] == ATypeTag.SERIALIZED_INT16_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT16);
+ short val = AInt16SerializerDeserializer.getShort(data, offset + 1);
+ aInt16.setValue(val);
+ serde.serialize(aInt16, out);
+ } else if (data[offset] == ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+ int val = AInt32SerializerDeserializer.getInt(data, offset + 1);
+ aInt32.setValue(val);
+ serde.serialize(aInt32, out);
+ } else if (data[offset] == ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+ long val = AInt64SerializerDeserializer.getLong(data, offset + 1);
+ aInt64.setValue(val);
+ serde.serialize(aInt64, out);
+ } else if (data[offset] == ATypeTag.SERIALIZED_FLOAT_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AFLOAT);
+ float val = AFloatSerializerDeserializer.getFloat(data, offset + 1);
+ if (Float.isNaN(val) || Float.isInfinite(val) || Float.compare(val, -0.0F) == 0
+ || Float.compare(val, 0.0F) == 0) {
+ aFloat.setValue(val);
+ serde.serialize(aFloat, out);
} else {
- throw new NotImplementedException(
- NumericTruncDescriptor.this.getIdentifier().getName() + ": not implemented for "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[offset]));
+ BigDecimal r = new BigDecimal(Float.toString(val));
+ aFloat.setValue(r.setScale(getPrecision(), BigDecimal.ROUND_DOWN).floatValue());
+ serde.serialize(aFloat, out);
}
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
+ } else if (data[offset] == ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+ double val = ADoubleSerializerDeserializer.getDouble(data, offset + 1);
+ if (Double.isNaN(val) || Double.isInfinite(val) || Double.compare(val, -0.0D) == 0
+ || Double.compare(val, 0.0D) == 0) {
+ aDouble.setValue(val);
+ serde.serialize(aDouble, out);
+ } else {
+ BigDecimal r = new BigDecimal(Double.toString(val));
+ aDouble.setValue(r.setScale(getPrecision(), BigDecimal.ROUND_DOWN).doubleValue());
+ serde.serialize(aDouble, out);
+ }
+ } else {
+ throw new TypeMismatchException(getIdentifier(), 0, data[offset],
+ ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
+ ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java
index 43b72c0..1478962 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java
@@ -22,11 +22,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
public class NumericUnaryMinusDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -50,7 +50,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new NumericUnaryMinusEvaluator(ctx, args[0]);
}
};
@@ -59,42 +59,42 @@
private class NumericUnaryMinusEvaluator extends AbstractUnaryNumericFunctionEval {
NumericUnaryMinusEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
- throws AlgebricksException {
+ throws HyracksDataException {
super(context, argEvalFactory, NumericUnaryMinusDescriptor.this.getIdentifier());
}
@Override
- protected void processInt8(byte arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt8(byte arg, IPointable resultPointable) throws HyracksDataException {
aInt8.setValue((byte) -arg);
serialize(aInt8, int8Serde, resultPointable);
}
@Override
- protected void processInt16(short arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt16(short arg, IPointable resultPointable) throws HyracksDataException {
aInt16.setValue((short) -arg);
serialize(aInt16, int16Serde, resultPointable);
}
@Override
- protected void processInt32(int arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt32(int arg, IPointable resultPointable) throws HyracksDataException {
aInt32.setValue(-arg);
serialize(aInt32, int32Serde, resultPointable);
}
@Override
- protected void processInt64(long arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processInt64(long arg, IPointable resultPointable) throws HyracksDataException {
aInt64.setValue(-arg);
serialize(aInt64, int64Serde, resultPointable);
}
@Override
- protected void processFloat(float arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processFloat(float arg, IPointable resultPointable) throws HyracksDataException {
aFloat.setValue(-arg);
serialize(aFloat, floatSerde, resultPointable);
}
@Override
- protected void processDouble(double arg, IPointable resultPointable) throws AlgebricksException {
+ protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
aDouble.setValue(-arg);
serialize(aDouble, doubleSerde, resultPointable);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrDescriptor.java
index a98f4bb..6c1ac38 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrDescriptor.java
@@ -31,7 +31,7 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -58,14 +58,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
-
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
final IPointable argPtr = new VoidPointable();
final IScalarEvaluator[] evals = new IScalarEvaluator[args.length];
for (int i = 0; i < evals.length; i++) {
@@ -86,49 +84,49 @@
.getSerializerDeserializer(BuiltinType.AMISSING);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
- try {
- resultStorage.reset();
- int n = args.length;
- boolean res = false;
- boolean metNull = false;
- boolean metMissing = false;
- for (int i = 0; i < n; i++) {
- evals[i].evaluate(tuple, argPtr);
- byte[] data = argPtr.getByteArray();
- int offset = argPtr.getStartOffset();
- if (data[offset] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
- metMissing = true;
- continue;
- }
- if (data[offset] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
- metNull = true;
- continue;
- }
- boolean argResult = ABooleanSerializerDeserializer.getBoolean(data, offset + 1);
- if (argResult == true) {
- // anything OR TRUE = TRUE
- booleanSerde.serialize(ABoolean.TRUE, output);
- result.set(resultStorage);
- return;
- }
- res |= argResult;
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ int n = args.length;
+ boolean res = false;
+ boolean metNull = false;
+ boolean metMissing = false;
+ for (int i = 0; i < n; i++) {
+ evals[i].evaluate(tuple, argPtr);
+ byte[] data = argPtr.getByteArray();
+ int offset = argPtr.getStartOffset();
+ if (data[offset] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
+ metMissing = true;
+ continue;
}
- if (metNull) {
- // NULL OR FALSE = NULL
- // NULL OR MISSING = NULL
- nullSerde.serialize(ANull.NULL, output);
- } else if (metMissing) {
- // MISSING OR FALSE = MISSING
- missingSerde.serialize(AMissing.MISSING, output);
- } else {
- ABoolean aResult = res ? ABoolean.TRUE : ABoolean.FALSE;
- booleanSerde.serialize(aResult, output);
+ if (data[offset] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
+ metNull = true;
+ continue;
}
- result.set(resultStorage);
- } catch (HyracksDataException hde) {
- throw new AlgebricksException(hde);
+ if (data[offset] != ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), i, data[offset],
+ ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
+ }
+ boolean argResult = ABooleanSerializerDeserializer.getBoolean(data, offset + 1);
+ if (argResult == true) {
+ // anything OR TRUE = TRUE
+ booleanSerde.serialize(ABoolean.TRUE, output);
+ result.set(resultStorage);
+ return;
+ }
+ res |= argResult;
}
+ if (metNull) {
+ // NULL OR FALSE = NULL
+ // NULL OR MISSING = NULL
+ nullSerde.serialize(ANull.NULL, output);
+ } else if (metMissing) {
+ // MISSING OR FALSE = MISSING
+ missingSerde.serialize(AMissing.MISSING, output);
+ } else {
+ ABoolean aResult = res ? ABoolean.TRUE : ABoolean.FALSE;
+ booleanSerde.serialize(aResult, output);
+ }
+ result.set(resultStorage);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrderedListConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrderedListConstructorDescriptor.java
index ffb8432..2dd9ef9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrderedListConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrderedListConstructorDescriptor.java
@@ -28,11 +28,11 @@
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -83,7 +83,7 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
final IScalarEvaluator[] argEvals = new IScalarEvaluator[args.length];
for (int i = 0; i < args.length; i++) {
argEvals[i] = args[i].createScalarEvaluator(ctx);
@@ -96,7 +96,7 @@
private final OrderedListBuilder builder = new OrderedListBuilder();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
builder.reset(orderedlistType);
@@ -108,11 +108,11 @@
builder.write(out, true);
result.set(resultStorage);
} catch (IOException ioe) {
- throw new AlgebricksException(ioe);
+ throw new HyracksDataException(ioe);
}
}
- private void writeUntypedItems(IFrameTupleReference tuple) throws AlgebricksException {
+ private void writeUntypedItems(IFrameTupleReference tuple) throws HyracksDataException {
try {
for (int i = 0; i < argEvals.length; i++) {
@@ -121,11 +121,11 @@
}
} catch (IOException ioe) {
- throw new AlgebricksException(ioe);
+ throw new HyracksDataException(ioe);
}
}
- private void writeTypedItems(IFrameTupleReference tuple) throws AlgebricksException {
+ private void writeTypedItems(IFrameTupleReference tuple) throws HyracksDataException {
try {
for (int i = 0; i < argEvals.length; i++) {
@@ -134,7 +134,7 @@
}
} catch (IOException ioe) {
- throw new AlgebricksException(ioe);
+ throw new HyracksDataException(ioe);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PointableHelper.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PointableHelper.java
index 24e66f2..5a293f3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PointableHelper.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PointableHelper.java
@@ -25,7 +25,6 @@
import org.apache.asterix.om.pointables.base.IVisitablePointable;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -112,7 +111,6 @@
* The storage buffer
* @param writeTag
* Specifying whether a tag for the string should also be written
- * @throws AlgebricksException
*/
public void serializeString(String str, IMutableValueStorage vs, boolean writeTag) throws AsterixException {
vs.reset();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java
index 8bfcfae..a4ba6dc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java
@@ -27,19 +27,19 @@
import org.apache.asterix.fuzzyjoin.similarity.SimilarityFilters;
import org.apache.asterix.om.base.AInt32;
import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.SimilarityFiltersCache;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.IntegerPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -59,14 +59,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
-
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
@@ -86,16 +84,15 @@
.getSerializerDeserializer(BuiltinType.AINT32);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
// length
evalLen.evaluate(tuple, inputVal);
byte[] data = inputVal.getByteArray();
int offset = inputVal.getStartOffset();
if (data[offset] != ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects type Int32 for the first argument, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[offset]));
+ throw new TypeMismatchException(getIdentifier(), 0, data[offset],
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG);
}
int length = IntegerPointable.getInteger(data, offset + 1);
@@ -104,9 +101,8 @@
data = inputVal.getByteArray();
offset = inputVal.getStartOffset();
if (data[offset] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects type DOUBLE for the second argument, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[offset]));
+ throw new TypeMismatchException(getIdentifier(), 1, data[offset],
+ ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
}
float similarityThreshold = (float) ADoubleSerializerDeserializer.getDouble(data, offset + 1);
@@ -116,9 +112,8 @@
offset = inputVal.getStartOffset();
int len = inputVal.getLength();
if (data[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects type STRING for the third argument, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[offset]));
+ throw new TypeMismatchException(getIdentifier(), 2, data[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
SimilarityFilters similarityFilters = similarityFiltersCache.get(similarityThreshold, data,
offset, len);
@@ -129,7 +124,7 @@
try {
int32Serde.serialize(res, out);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java
index c9e5807..bb3a522 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java
@@ -31,10 +31,9 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -57,14 +56,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
-
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
@@ -85,26 +82,20 @@
.getSerializerDeserializer(BuiltinType.AINT32);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
evalLen.evaluate(tuple, lenPtr);
evalThreshold.evaluate(tuple, thresholdPtr);
// length
- int length = 0;
- try {
- length = ATypeHierarchy.getIntegerValue(lenPtr.getByteArray(), lenPtr.getStartOffset());
- } catch (HyracksDataException e1) {
- throw new AlgebricksException(e1);
- }
-
+ int length = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, lenPtr.getByteArray(),
+ lenPtr.getStartOffset());
// similarity threshold
byte[] data = thresholdPtr.getByteArray();
int offset = thresholdPtr.getStartOffset();
if (data[offset] != ATypeTag.SERIALIZED_FLOAT_TYPE_TAG) {
- throw new AlgebricksException(AsterixBuiltinFunctions.PREFIX_LEN_JACCARD.getName()
- + ": expects type FLOAT the first argument but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[offset]));
+ throw new TypeMismatchException(getIdentifier(), 1, data[offset],
+ ATypeTag.SERIALIZED_FLOAT_TYPE_TAG);
}
float similarityThreshold = AFloatSerializerDeserializer.getFloat(data, offset + 1);
@@ -118,7 +109,7 @@
try {
int32Serde.serialize(res, out);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
index d1d0a3c..5f29fad 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
@@ -33,19 +33,19 @@
import org.apache.asterix.fuzzyjoin.similarity.SimilarityMetric;
import org.apache.asterix.om.base.ADouble;
import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.SimilarityFiltersCache;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.IntegerPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -65,14 +65,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
-
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
@@ -100,7 +98,7 @@
.getSerializerDeserializer(BuiltinType.ADOUBLE);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
// similarity threshold
evalThreshold.evaluate(tuple, inputVal);
@@ -108,9 +106,8 @@
int offset = inputVal.getStartOffset();
if (data[offset] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects type DOUBLE for the first argument but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[offset]));
+ throw new TypeMismatchException(getIdentifier(), 0, data[offset],
+ ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
}
float similarityThreshold = (float) ADoubleSerializerDeserializer.getDouble(data, offset + 1);
@@ -120,9 +117,8 @@
offset = inputVal.getStartOffset();
int len = inputVal.getLength();
if (data[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects type STRING for the second argument but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[offset]));
+ throw new TypeMismatchException(getIdentifier(), 1, data[offset],
+ ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
}
SimilarityFilters similarityFilters = similarityFiltersCache.get(similarityThreshold, data,
offset, len);
@@ -131,9 +127,8 @@
data = inputVal.getByteArray();
offset = inputVal.getStartOffset();
if (data[offset] != ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects type INT32 for the thrid argument but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[offset]));
+ throw new TypeMismatchException(getIdentifier(), 2, data[offset],
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG);
}
int length1 = IntegerPointable.getInteger(data, offset + 1);
@@ -141,9 +136,8 @@
data = inputVal.getByteArray();
offset = inputVal.getStartOffset();
if (data[offset] != ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects type INT32 for the fourth argument but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[offset]));
+ throw new TypeMismatchException(getIdentifier(), 3, data[offset],
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG);
}
int length2 = IntegerPointable.getInteger(data, offset + 1);
@@ -163,8 +157,9 @@
if (serList[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
&& serList[offset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
- throw new AlgebricksException(FID.getName() + ": not defined for values of type"
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serList[offset]));
+ throw new TypeMismatchException(getIdentifier(), 4, data[offset],
+ ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
+ ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
}
int lengthTokens1;
@@ -177,7 +172,7 @@
itemOffset = AOrderedListSerializerDeserializer.getItemOffset(serList, offset,
i);
} catch (AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
tokens1.add(IntegerPointable.getInteger(serList, itemOffset));
}
@@ -190,7 +185,7 @@
itemOffset = AUnorderedListSerializerDeserializer.getItemOffset(serList, offset,
i);
} catch (AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
tokens1.add(IntegerPointable.getInteger(serList, itemOffset));
}
@@ -208,8 +203,9 @@
if (serList[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
&& serList[offset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
- throw new AlgebricksException(FID.getName() + ": not defined for values of type"
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serList[offset]));
+ throw new TypeMismatchException(getIdentifier(), 5, data[offset],
+ ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
+ ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
}
int lengthTokens2;
@@ -222,7 +218,7 @@
itemOffset = AOrderedListSerializerDeserializer.getItemOffset(serList, offset,
i);
} catch (AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
tokens2.add(IntegerPointable.getInteger(serList, itemOffset));
}
@@ -235,7 +231,7 @@
itemOffset = AUnorderedListSerializerDeserializer.getItemOffset(serList, offset,
i);
} catch (AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
tokens2.add(IntegerPointable.getInteger(serList, itemOffset));
}
@@ -276,7 +272,7 @@
try {
doubleSerde.serialize(res, out);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardCheckDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardCheckDescriptor.java
index 29a6f6f..fabe832 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardCheckDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardCheckDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.SimilarityJaccardCheckEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SimilarityJaccardCheckDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -40,13 +40,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new SimilarityJaccardCheckEvaluator(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardDescriptor.java
index 6c3847a..e81143d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.SimilarityJaccardEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SimilarityJaccardDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -40,13 +40,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new SimilarityJaccardEvaluator(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixCheckDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixCheckDescriptor.java
index e28a784..97999ef 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixCheckDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixCheckDescriptor.java
@@ -32,12 +32,12 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.SimilarityJaccardPrefixEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
public class SimilarityJaccardPrefixCheckDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -52,13 +52,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new SimilarityJaccardPrefixCheckEvaluator(args, ctx);
}
};
@@ -84,14 +83,14 @@
private final AOrderedListType listType = new AOrderedListType(BuiltinType.ANY, "list");
public SimilarityJaccardPrefixCheckEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
- throws AlgebricksException {
+ throws HyracksDataException {
super(args, context);
listBuilder = new OrderedListBuilder();
inputVal = new ArrayBackedValueStorage();
}
@Override
- public void writeResult() throws AlgebricksException, IOException {
+ public void writeResult() throws HyracksDataException, IOException {
listBuilder.reset(listType);
boolean matches = (sim <= 0) ? false : true;
float jaccSim = (matches) ? sim : 0.0f;
@@ -104,7 +103,6 @@
aFloat.setValue(jaccSim);
floatSerde.serialize(aFloat, inputVal.getDataOutput());
listBuilder.addItem(inputVal);
-
listBuilder.write(out, true);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixDescriptor.java
index 144a328..cb479d5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.SimilarityJaccardPrefixEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SimilarityJaccardPrefixDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -40,13 +40,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new SimilarityJaccardPrefixEvaluator(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardSortedCheckDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardSortedCheckDescriptor.java
index 6e34ffb..9c1cddb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardSortedCheckDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardSortedCheckDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.SimilarityJaccardSortedCheckEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
// Assumes that both arguments are sorted by the same ordering.
public class SimilarityJaccardSortedCheckDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -41,13 +41,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new SimilarityJaccardSortedCheckEvaluator(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardSortedDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardSortedDescriptor.java
index 4995e25..47195e2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardSortedDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardSortedDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.SimilarityJaccardSortedEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
// Assumes that both arguments are sorted by the same ordering.
public class SimilarityJaccardSortedDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -41,13 +41,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new SimilarityJaccardSortedEvaluator(args, ctx);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java
index e31ad90..dec2cdb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java
@@ -33,8 +33,8 @@
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.SpatialUtils;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -56,13 +56,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -71,7 +70,7 @@
private final IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval.evaluate(tuple, argPtr);
@@ -86,8 +85,8 @@
int numOfPoints = AInt16SerializerDeserializer.getShort(bytes, offset + 1);
if (numOfPoints < 3) {
- throw new AlgebricksException(AsterixBuiltinFunctions.SPATIAL_AREA.getName()
- + ": polygon must have at least 3 points");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
}
area = Math.abs(SpatialUtils.polygonArea(bytes, offset, numOfPoints));
out.writeByte(ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
@@ -119,14 +118,12 @@
out.writeDouble(area);
break;
default:
- throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_AREA.getName()
- + ": does not support the type: " + tag
- + "; it is only implemented for POLYGON, CIRCLE and RECTANGLE.");
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG, ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
}
- } catch (HyracksDataException hde) {
- throw new AlgebricksException(hde);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java
index 2208d6f..1862517 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java
@@ -33,14 +33,14 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -57,13 +57,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -84,7 +83,7 @@
.getSerializerDeserializer(BuiltinType.ARECTANGLE);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, inputArg0);
eval1.evaluate(tuple, inputArg1);
@@ -101,12 +100,13 @@
int offset3 = inputArg3.getStartOffset();
try {
- ATypeTag tag0 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
- ATypeTag tag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
- ATypeTag tag2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes2[offset2]);
- ATypeTag tag3 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes3[offset3]);
- if (tag0 == ATypeTag.POINT && tag1 == ATypeTag.POINT && tag2 == ATypeTag.DOUBLE
- && tag3 == ATypeTag.DOUBLE) {
+ byte tag0 = bytes0[offset0];
+ byte tag1 = bytes1[offset1];
+ byte tag2 = bytes2[offset2];
+ byte tag3 = bytes3[offset3];
+ if (tag0 == ATypeTag.SERIALIZED_POINT_TYPE_TAG && tag1 == ATypeTag.SERIALIZED_POINT_TYPE_TAG
+ && tag2 == ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG
+ && tag3 == ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
double xLoc = ADoubleSerializerDeserializer.getDouble(bytes0,
offset0 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
double yLoc = ADoubleSerializerDeserializer.getDouble(bytes0,
@@ -127,16 +127,26 @@
aRectangle.setValue(aPoint[0], aPoint[1]);
rectangleSerde.serialize(aRectangle, out);
} else {
- throw new AlgebricksException(AsterixBuiltinFunctions.SPATIAL_CELL.getName()
- + ": expects input type: (POINT, POINT, DOUBLE, DOUBLE) but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes2[offset2]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes3[offset3]) + ").");
+ if (tag0 != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, tag0,
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG);
+ }
+ if (tag1 != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, tag1,
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG);
+ }
+ if (tag2 != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 2, tag2,
+ ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+ }
+ if (tag3 != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 3, tag3,
+ ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+ }
}
result.set(resultStorage);
} catch (IOException e1) {
- throw new AlgebricksException(e1);
+ throw new HyracksDataException(e1);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java
index 5c969b1..0ff1eb1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java
@@ -24,14 +24,12 @@
import org.apache.asterix.dataflow.data.nontagged.Coordinate;
import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.APointSerializerDeserializer;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -53,13 +51,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -70,7 +67,7 @@
private final IScalarEvaluator eval1 = args[1].createScalarEvaluator(ctx);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, inputArg0);
eval1.evaluate(tuple, inputArg1);
@@ -81,11 +78,11 @@
int offset0 = inputArg0.getStartOffset();
int offset1 = inputArg1.getStartOffset();
- ATypeTag tag0 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
- ATypeTag tag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
+ byte tag0 = bytes0[offset0];
+ byte tag1 = bytes1[offset1];
double distance;
- if (tag0 == ATypeTag.POINT) {
- if (tag1 == ATypeTag.POINT) {
+ if (tag0 == ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
+ if (tag1 == ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
double x1 = ADoubleSerializerDeserializer.getDouble(bytes0,
offset0 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
double y1 = ADoubleSerializerDeserializer.getDouble(bytes0,
@@ -96,21 +93,17 @@
offset1 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
distance = Math.sqrt(Math.pow(x2 - x1, 2) + Math.pow(y2 - y1, 2));
} else {
- throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_DISTANCE.getName()
- + ": does not support the type: " + tag1
- + "; it is only implemented for POINT.");
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG);
}
} else {
- throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_DISTANCE.getName()
- + ": does not support the type: " + tag1
- + "; it is only implemented for POINT.");
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG);
}
out.writeByte(ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
out.writeDouble(distance);
- } catch (HyracksDataException hde) {
- throw new AlgebricksException(hde);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
index 03ec9a6..e4704f2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
@@ -40,8 +40,8 @@
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.DoubleArray;
import org.apache.asterix.runtime.evaluators.common.SpatialUtils;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -65,13 +65,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -123,8 +122,7 @@
offset1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
if (numOfPoints1 < 3) {
- throw new HyracksDataException(AsterixBuiltinFunctions.SPATIAL_INTERSECT.getName()
- + ": polygon must have at least 3 points.");
+ throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
}
int counter = 0;
@@ -233,8 +231,7 @@
offset1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
if (numOfPoints1 < 3) {
- throw new HyracksDataException(AsterixBuiltinFunctions.SPATIAL_INTERSECT.getName()
- + ": polygon must have at least 3 points.");
+ throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
}
for (int i = 0; i < numOfPoints1; i++) {
double startX2 = ADoubleSerializerDeserializer.getDouble(bytes1,
@@ -386,8 +383,8 @@
int v = middleVertex;
while (!foundEar) {
if (0 >= (nonSimplePolygonDetection--)) {
- throw new HyracksDataException(AsterixBuiltinFunctions.SPATIAL_INTERSECT.getName()
- + ": non-simple polygons are not supported.");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
}
int u = v;
if (numOfPoints <= u) {
@@ -674,13 +671,12 @@
}
private boolean rectanglePolygonIntersection(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
- throws HyracksDataException, AlgebricksException {
+ throws HyracksDataException {
int numOfPoints1 = AInt16SerializerDeserializer.getShort(bytes1,
offset1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
if (numOfPoints1 < 3) {
- throw new HyracksDataException(AsterixBuiltinFunctions.SPATIAL_INTERSECT.getName()
- + ": polygon must have at least 3 points.");
+ throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
}
getCounterClockWisePolygon(bytes1, offset1, pointsOffsets1, numOfPoints1);
@@ -727,13 +723,12 @@
}
private boolean polygonCircleIntersection(byte[] bytes0, int offset0, byte[] bytes1, int offset1)
- throws HyracksDataException, AlgebricksException {
+ throws HyracksDataException {
int numOfPoints = AInt16SerializerDeserializer.getShort(bytes0,
offset0 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
if (numOfPoints < 3) {
- throw new HyracksDataException(AsterixBuiltinFunctions.SPATIAL_INTERSECT.getName()
- + ": polygon must have at least 3 points.");
+ throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
}
getCounterClockWisePolygon(bytes0, offset0, pointsOffsets0, numOfPoints);
@@ -765,7 +760,7 @@
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, inputArg0);
eval1.evaluate(tuple, inputArg1);
@@ -775,305 +770,313 @@
int offset0 = inputArg0.getStartOffset();
int offset1 = inputArg1.getStartOffset();
- try {
- boolean res = false;
- ATypeTag tag0 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
- ATypeTag tag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
+ boolean res = false;
+ ATypeTag tag0 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
+ ATypeTag tag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
- switch (tag0) {
- case POINT:
- switch (tag1) {
- case POINT:
+ switch (tag0) {
+ case POINT:
+ switch (tag1) {
+ case POINT:
+ if (ascDoubleComp.compare(bytes0,
+ offset0 + APointSerializerDeserializer
+ .getCoordinateOffset(Coordinate.X),
+ 8, bytes1, offset1 + APointSerializerDeserializer
+ .getCoordinateOffset(Coordinate.X),
+ 8) == 0) {
if (ascDoubleComp.compare(bytes0,
offset0 + APointSerializerDeserializer
- .getCoordinateOffset(Coordinate.X),
+ .getCoordinateOffset(Coordinate.Y),
8, bytes1, offset1 + APointSerializerDeserializer
- .getCoordinateOffset(Coordinate.X),
+ .getCoordinateOffset(Coordinate.Y),
8) == 0) {
- if (ascDoubleComp.compare(bytes0,
- offset0 + APointSerializerDeserializer
- .getCoordinateOffset(Coordinate.Y),
- 8, bytes1, offset1 + APointSerializerDeserializer
- .getCoordinateOffset(Coordinate.Y),
- 8) == 0) {
- res = true;
- }
+ res = true;
}
- break;
- case LINE:
- double pX = ADoubleSerializerDeserializer.getDouble(bytes0, offset0
- + APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
- double pY = ADoubleSerializerDeserializer.getDouble(bytes0, offset0
- + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
+ }
+ break;
+ case LINE:
+ double pX = ADoubleSerializerDeserializer.getDouble(bytes0, offset0
+ + APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
+ double pY = ADoubleSerializerDeserializer.getDouble(bytes0, offset0
+ + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
- double startX = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + ALineSerializerDeserializer
- .getStartPointCoordinateOffset(Coordinate.X));
- double startY = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + ALineSerializerDeserializer
- .getStartPointCoordinateOffset(Coordinate.Y));
- double endX = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + ALineSerializerDeserializer
- .getEndPointCoordinateOffset(Coordinate.X));
- double endY = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + ALineSerializerDeserializer
- .getEndPointCoordinateOffset(Coordinate.Y));
+ double startX = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + ALineSerializerDeserializer
+ .getStartPointCoordinateOffset(Coordinate.X));
+ double startY = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + ALineSerializerDeserializer
+ .getStartPointCoordinateOffset(Coordinate.Y));
+ double endX = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + ALineSerializerDeserializer
+ .getEndPointCoordinateOffset(Coordinate.X));
+ double endY = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + ALineSerializerDeserializer
+ .getEndPointCoordinateOffset(Coordinate.Y));
- res = pointOnLine(pX, pY, startX, startY, endX, endY);
- break;
- case POLYGON:
- res = pointInPolygon(bytes0, offset0, bytes1, offset1);
- break;
- case CIRCLE:
- res = pointInCircle(bytes0, offset0, bytes1, offset1);
- break;
- case RECTANGLE:
- res = pointInRectangle(bytes0, offset0, bytes1, offset1);
- break;
- default:
- throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_INTERSECT
- .getName() + ": does not support the type: " + tag1
- + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
- }
- break;
- case LINE:
- switch (tag1) {
- case POINT:
- double pX = ADoubleSerializerDeserializer.getDouble(bytes1, offset1
- + APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
- double pY = ADoubleSerializerDeserializer.getDouble(bytes1, offset1
- + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
+ res = pointOnLine(pX, pY, startX, startY, endX, endY);
+ break;
+ case POLYGON:
+ res = pointInPolygon(bytes0, offset0, bytes1, offset1);
+ break;
+ case CIRCLE:
+ res = pointInCircle(bytes0, offset0, bytes1, offset1);
+ break;
+ case RECTANGLE:
+ res = pointInRectangle(bytes0, offset0, bytes1, offset1);
+ break;
+ default:
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
+ ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+ }
+ break;
+ case LINE:
+ switch (tag1) {
+ case POINT:
+ double pX = ADoubleSerializerDeserializer.getDouble(bytes1, offset1
+ + APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
+ double pY = ADoubleSerializerDeserializer.getDouble(bytes1, offset1
+ + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
- double startX = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + ALineSerializerDeserializer
- .getStartPointCoordinateOffset(Coordinate.X));
- double startY = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + ALineSerializerDeserializer
- .getStartPointCoordinateOffset(Coordinate.Y));
- double endX = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + ALineSerializerDeserializer
- .getEndPointCoordinateOffset(Coordinate.X));
- double endY = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + ALineSerializerDeserializer
- .getEndPointCoordinateOffset(Coordinate.Y));
+ double startX = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + ALineSerializerDeserializer
+ .getStartPointCoordinateOffset(Coordinate.X));
+ double startY = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + ALineSerializerDeserializer
+ .getStartPointCoordinateOffset(Coordinate.Y));
+ double endX = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + ALineSerializerDeserializer
+ .getEndPointCoordinateOffset(Coordinate.X));
+ double endY = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + ALineSerializerDeserializer
+ .getEndPointCoordinateOffset(Coordinate.Y));
- res = pointOnLine(pX, pY, startX, startY, endX, endY);
- break;
- case LINE:
- double startX1 = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + ALineSerializerDeserializer
- .getStartPointCoordinateOffset(Coordinate.X));
- double startY1 = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + ALineSerializerDeserializer
- .getStartPointCoordinateOffset(Coordinate.Y));
- double endX1 = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + ALineSerializerDeserializer
- .getEndPointCoordinateOffset(Coordinate.X));
- double endY1 = ADoubleSerializerDeserializer.getDouble(bytes0,
- offset0 + ALineSerializerDeserializer
- .getEndPointCoordinateOffset(Coordinate.Y));
+ res = pointOnLine(pX, pY, startX, startY, endX, endY);
+ break;
+ case LINE:
+ double startX1 = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + ALineSerializerDeserializer
+ .getStartPointCoordinateOffset(Coordinate.X));
+ double startY1 = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + ALineSerializerDeserializer
+ .getStartPointCoordinateOffset(Coordinate.Y));
+ double endX1 = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + ALineSerializerDeserializer
+ .getEndPointCoordinateOffset(Coordinate.X));
+ double endY1 = ADoubleSerializerDeserializer.getDouble(bytes0,
+ offset0 + ALineSerializerDeserializer
+ .getEndPointCoordinateOffset(Coordinate.Y));
- double startX2 = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + ALineSerializerDeserializer
- .getStartPointCoordinateOffset(Coordinate.X));
- double startY2 = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + ALineSerializerDeserializer
- .getStartPointCoordinateOffset(Coordinate.Y));
- double endX2 = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + ALineSerializerDeserializer
- .getEndPointCoordinateOffset(Coordinate.X));
- double endY2 = ADoubleSerializerDeserializer.getDouble(bytes1,
- offset1 + ALineSerializerDeserializer
- .getEndPointCoordinateOffset(Coordinate.Y));
- res = lineLineIntersection(startX1, startY1, endX1, endY1, startX2, startY2,
- endX2, endY2);
- break;
- case POLYGON:
- res = linePolygonIntersection(bytes0, offset0, bytes1, offset1);
- break;
- case CIRCLE:
- res = lineCircleIntersection(bytes0, offset0, bytes1, offset1);
- break;
- case RECTANGLE:
- res = lineRectangleIntersection(bytes0, offset0, bytes1, offset1);
- break;
- default:
- throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_INTERSECT
- .getName() + ": does not support the type: " + tag1
- + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
- }
- break;
- case POLYGON:
- switch (tag1) {
- case POINT:
- res = pointInPolygon(bytes1, offset1, bytes0, offset0);
- break;
- case LINE:
- res = linePolygonIntersection(bytes1, offset1, bytes0, offset0);
- break;
- case POLYGON:
- int numOfPoints0 = AInt16SerializerDeserializer.getShort(bytes0,
- offset0 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
- int numOfPoints1 = AInt16SerializerDeserializer.getShort(bytes1,
- offset1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
+ double startX2 = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + ALineSerializerDeserializer
+ .getStartPointCoordinateOffset(Coordinate.X));
+ double startY2 = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + ALineSerializerDeserializer
+ .getStartPointCoordinateOffset(Coordinate.Y));
+ double endX2 = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + ALineSerializerDeserializer
+ .getEndPointCoordinateOffset(Coordinate.X));
+ double endY2 = ADoubleSerializerDeserializer.getDouble(bytes1,
+ offset1 + ALineSerializerDeserializer
+ .getEndPointCoordinateOffset(Coordinate.Y));
+ res = lineLineIntersection(startX1, startY1, endX1, endY1, startX2, startY2,
+ endX2, endY2);
+ break;
+ case POLYGON:
+ res = linePolygonIntersection(bytes0, offset0, bytes1, offset1);
+ break;
+ case CIRCLE:
+ res = lineCircleIntersection(bytes0, offset0, bytes1, offset1);
+ break;
+ case RECTANGLE:
+ res = lineRectangleIntersection(bytes0, offset0, bytes1, offset1);
+ break;
+ default:
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
+ ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+ }
+ break;
+ case POLYGON:
+ switch (tag1) {
+ case POINT:
+ res = pointInPolygon(bytes1, offset1, bytes0, offset0);
+ break;
+ case LINE:
+ res = linePolygonIntersection(bytes1, offset1, bytes0, offset0);
+ break;
+ case POLYGON:
+ int numOfPoints0 = AInt16SerializerDeserializer.getShort(bytes0,
+ offset0 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
+ int numOfPoints1 = AInt16SerializerDeserializer.getShort(bytes1,
+ offset1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
- if (numOfPoints0 < 3 || numOfPoints1 < 3) {
- throw new AlgebricksException("Polygon must have at least 3 points.");
+ if (numOfPoints0 < 3 || numOfPoints1 < 3) {
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
+ }
+
+ getCounterClockWisePolygon(bytes0, offset0, pointsOffsets0, numOfPoints0);
+ getCounterClockWisePolygon(bytes1, offset1, pointsOffsets1, numOfPoints1);
+ int nonSimplePolygonDetection0 = 2 * numOfPoints0;
+ int nonSimplePolygonDetection1 = 2 * numOfPoints1;
+ boolean intersect = false;
+ int middleVertex0 = numOfPoints0 - 1;
+
+ int numOfTriangles1 = 0;
+ int middleVertex1 = numOfPoints1 - 1;
+ trianglesX1.reset();
+ trianglesY1.reset();
+ while (true) {
+ middleVertex1 = triangulatePolygon(bytes1, offset1, numOfPoints1,
+ pointsOffsets1, trianglesX1, trianglesY1, numOfTriangles1,
+ nonSimplePolygonDetection1, middleVertex1);
+
+ if (middleVertex1 == -1) {
+ break;
}
- getCounterClockWisePolygon(bytes0, offset0, pointsOffsets0, numOfPoints0);
- getCounterClockWisePolygon(bytes1, offset1, pointsOffsets1, numOfPoints1);
- int nonSimplePolygonDetection0 = 2 * numOfPoints0;
- int nonSimplePolygonDetection1 = 2 * numOfPoints1;
- boolean intersect = false;
- int middleVertex0 = numOfPoints0 - 1;
+ numOfPoints1--;
+ nonSimplePolygonDetection1 = 2 * numOfPoints1;
+ numOfTriangles1++;
+ }
+ int numOfTriangles0 = 0;
+ trianglesX0.reset();
+ trianglesY0.reset();
+ while (true) {
+ middleVertex0 = triangulatePolygon(bytes0, offset0, numOfPoints0,
+ pointsOffsets0, trianglesX0, trianglesY0, numOfTriangles0,
+ nonSimplePolygonDetection0, middleVertex0);
- int numOfTriangles1 = 0;
- int middleVertex1 = numOfPoints1 - 1;
- trianglesX1.reset();
- trianglesY1.reset();
- while (true) {
- middleVertex1 = triangulatePolygon(bytes1, offset1, numOfPoints1,
- pointsOffsets1, trianglesX1, trianglesY1, numOfTriangles1,
- nonSimplePolygonDetection1, middleVertex1);
-
- if (middleVertex1 == -1) {
- break;
- }
-
- numOfPoints1--;
- nonSimplePolygonDetection1 = 2 * numOfPoints1;
- numOfTriangles1++;
+ if (middleVertex0 == -1) {
+ break;
}
- int numOfTriangles0 = 0;
- trianglesX0.reset();
- trianglesY0.reset();
- while (true) {
- middleVertex0 = triangulatePolygon(bytes0, offset0, numOfPoints0,
- pointsOffsets0, trianglesX0, trianglesY0, numOfTriangles0,
- nonSimplePolygonDetection0, middleVertex0);
+ numOfPoints0--;
+ nonSimplePolygonDetection0 = 2 * numOfPoints0;
+ numOfTriangles0++;
+ int lastTriangle = (trianglesX0.length() / 3) - 1;
- if (middleVertex0 == -1) {
- break;
- }
- numOfPoints0--;
- nonSimplePolygonDetection0 = 2 * numOfPoints0;
- numOfTriangles0++;
- int lastTriangle = (trianglesX0.length() / 3) - 1;
+ for (int i = 0; i < numOfTriangles1; i++) {
- for (int i = 0; i < numOfTriangles1; i++) {
+ res = triangleTriangleIntersection(trianglesX0, trianglesY0,
+ lastTriangle, trianglesX1, trianglesY1, i);
- res = triangleTriangleIntersection(trianglesX0, trianglesY0,
- lastTriangle, trianglesX1, trianglesY1, i);
-
- if (res) {
- res = triangleTriangleIntersection(trianglesX1, trianglesY1, i,
- trianglesX0, trianglesY0, lastTriangle);
-
- if (res) {
- intersect = true;
- break;
- }
- }
- }
- if (intersect) {
- break;
- }
- }
- break;
- case CIRCLE:
- res = polygonCircleIntersection(bytes0, offset0, bytes1, offset1);
- break;
- case RECTANGLE:
- res = rectanglePolygonIntersection(bytes1, offset1, bytes0, offset0);
- break;
- default:
- throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_INTERSECT
- .getName() + ": does not support the type: " + tag1
- + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
- }
- break;
- case CIRCLE:
- switch (tag1) {
- case POINT:
- res = pointInCircle(bytes0, offset0, bytes1, offset1);
- break;
- case LINE:
- res = lineCircleIntersection(bytes1, offset1, bytes0, offset0);
- break;
- case POLYGON:
- res = polygonCircleIntersection(bytes1, offset1, bytes0, offset0);
- break;
- case CIRCLE:
- res = circleCircleIntersection(bytes0, offset0, bytes1, offset1);
- break;
- case RECTANGLE:
- res = rectangleCircleIntersection(bytes1, offset1, bytes0, offset0);
- break;
- default:
- throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_INTERSECT
- .getName() + ": does not support the type: " + tag1
- + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
- }
- break;
- case RECTANGLE:
- switch (tag1) {
- case POINT:
- res = pointInRectangle(bytes1, offset1, bytes0, offset0);
- break;
- case LINE:
- res = lineRectangleIntersection(bytes1, offset1, bytes0, offset0);
- break;
- case POLYGON:
- res = rectanglePolygonIntersection(bytes0, offset0, bytes1, offset1);
- break;
- case CIRCLE:
- res = rectangleCircleIntersection(bytes0, offset0, bytes1, offset1);
- break;
- case RECTANGLE:
- triangulateRectangle(bytes0, offset0, trianglesX0, trianglesY0);
- triangulateRectangle(bytes1, offset1, trianglesX1, trianglesY1);
-
- boolean intersect = false;
- // 2 triangles in a rectangle
- for (int j = 0; j < 2; j++) {
- for (int i = 0; i < 2; i++) {
-
+ if (res) {
res = triangleTriangleIntersection(trianglesX1, trianglesY1, i,
- trianglesX0, trianglesY0, j);
+ trianglesX0, trianglesY0, lastTriangle);
if (res) {
- res = triangleTriangleIntersection(trianglesX0, trianglesY0, j,
- trianglesX1, trianglesY1, i);
-
- if (res) {
- intersect = true;
- break;
- }
+ intersect = true;
+ break;
}
}
- if (intersect) {
- break;
+ }
+ if (intersect) {
+ break;
+ }
+ }
+ break;
+ case CIRCLE:
+ res = polygonCircleIntersection(bytes0, offset0, bytes1, offset1);
+ break;
+ case RECTANGLE:
+ res = rectanglePolygonIntersection(bytes1, offset1, bytes0, offset0);
+ break;
+ default:
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
+ ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+ }
+ break;
+ case CIRCLE:
+ switch (tag1) {
+ case POINT:
+ res = pointInCircle(bytes0, offset0, bytes1, offset1);
+ break;
+ case LINE:
+ res = lineCircleIntersection(bytes1, offset1, bytes0, offset0);
+ break;
+ case POLYGON:
+ res = polygonCircleIntersection(bytes1, offset1, bytes0, offset0);
+ break;
+ case CIRCLE:
+ res = circleCircleIntersection(bytes0, offset0, bytes1, offset1);
+ break;
+ case RECTANGLE:
+ res = rectangleCircleIntersection(bytes1, offset1, bytes0, offset0);
+ break;
+ default:
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
+ ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+ }
+ break;
+ case RECTANGLE:
+ switch (tag1) {
+ case POINT:
+ res = pointInRectangle(bytes1, offset1, bytes0, offset0);
+ break;
+ case LINE:
+ res = lineRectangleIntersection(bytes1, offset1, bytes0, offset0);
+ break;
+ case POLYGON:
+ res = rectanglePolygonIntersection(bytes0, offset0, bytes1, offset1);
+ break;
+ case CIRCLE:
+ res = rectangleCircleIntersection(bytes0, offset0, bytes1, offset1);
+ break;
+ case RECTANGLE:
+ triangulateRectangle(bytes0, offset0, trianglesX0, trianglesY0);
+ triangulateRectangle(bytes1, offset1, trianglesX1, trianglesY1);
+
+ boolean intersect = false;
+ // 2 triangles in a rectangle
+ for (int j = 0; j < 2; j++) {
+ for (int i = 0; i < 2; i++) {
+
+ res = triangleTriangleIntersection(trianglesX1, trianglesY1, i,
+ trianglesX0, trianglesY0, j);
+
+ if (res) {
+ res = triangleTriangleIntersection(trianglesX0, trianglesY0, j,
+ trianglesX1, trianglesY1, i);
+
+ if (res) {
+ intersect = true;
+ break;
+ }
}
}
- break;
- default:
- throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_INTERSECT
- .getName() + ": does not support the type: " + tag1
- + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
- }
- break;
- default:
- throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_INTERSECT
- .getName() + ": does not support the type: " + tag1
- + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
- }
-
- ABoolean aResult = res ? (ABoolean.TRUE) : (ABoolean.FALSE);
- aBooleanSerDer.serialize(aResult, out);
- } catch (HyracksDataException hde) {
- throw new AlgebricksException(hde);
+ if (intersect) {
+ break;
+ }
+ }
+ break;
+ default:
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
+ ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+ }
+ break;
+ default:
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
+ ATypeTag.SERIALIZED_POLYGON_TYPE_TAG, ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
}
+
+ ABoolean aResult = res ? ABoolean.TRUE : ABoolean.FALSE;
+ aBooleanSerDer.serialize(aResult, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringConcatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringConcatDescriptor.java
index 07a79c4..1936902 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringConcatDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringConcatDescriptor.java
@@ -23,21 +23,23 @@
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMissing;
import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.AsterixListAccessor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -57,11 +59,10 @@
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -73,10 +74,12 @@
@SuppressWarnings("unchecked")
private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.ANULL);
+ private ISerializerDeserializer<AMissing> missingSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AMISSING);
private final byte[] tempLengthArray = new byte[5];
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
try {
evalList.evaluate(tuple, inputArgList);
@@ -85,15 +88,11 @@
if (listBytes[listOffset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
&& listBytes[listOffset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
- throw new AlgebricksException(AsterixBuiltinFunctions.STRING_CONCAT.getName()
- + ": expects input type ORDEREDLIST/UNORDEREDLIST, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(listBytes[listOffset]));
+ throw new TypeMismatchException(getIdentifier(), 0, listBytes[listOffset],
+ ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
+ ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
}
- try {
- listAccessor.reset(listBytes, listOffset);
- } catch (AsterixException e) {
- throw new AlgebricksException(e);
- }
+ listAccessor.reset(listBytes, listOffset);
try {
// calculate length first
int utf8Len = 0;
@@ -111,15 +110,18 @@
result.set(resultStorage);
return;
}
- throw new AlgebricksException(AsterixBuiltinFunctions.STRING_CONCAT.getName()
- + ": expects type STRING/NULL for the list item but got " + itemType);
+ if (itemType == ATypeTag.MISSING) {
+ missingSerde.serialize(AMissing.MISSING, out);
+ result.set(resultStorage);
+ return;
+ }
+ throw new UnsupportedItemTypeException(getIdentifier(), itemType.serialize());
}
utf8Len += UTF8StringUtil.getUTFLength(listBytes, itemOffset);
}
out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
int cbytes = UTF8StringUtil.encodeUTF8Length(utf8Len, tempLengthArray, 0);
out.write(tempLengthArray, 0, cbytes);
-
for (int i = 0; i < listAccessor.size(); i++) {
int itemOffset = listAccessor.getItemOffset(i);
if (listAccessor.itemsAreSelfDescribing()) {
@@ -130,10 +132,10 @@
utf8Len);
}
} catch (AsterixException ex) {
- throw new AlgebricksException(ex);
+ throw new HyracksDataException(ex);
}
} catch (IOException e1) {
- throw new AlgebricksException(e1.getMessage());
+ throw new HyracksDataException(e1);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java
index 5d88b6c..7786f0e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java
@@ -24,11 +24,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
public class StringContainsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -42,14 +42,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
-
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractBinaryStringBoolEval(ctx, args[0], args[1],
AsterixBuiltinFunctions.STRING_CONTAINS) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java
index c0996b7..dec422a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java
@@ -24,11 +24,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
public class StringEndsWithDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -42,14 +42,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
-
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractBinaryStringBoolEval(ctx, args[0], args[1],
AsterixBuiltinFunctions.STRING_ENDS_WITH) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
index 78fdba6..bcbf207 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
@@ -24,11 +24,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
public class StringEqualDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -42,14 +42,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
-
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractBinaryStringBoolEval(ctx, args[0], args[1],
AsterixBuiltinFunctions.STRING_EQUAL) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringInitCapDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringInitCapDescriptor.java
index fee7e06..930f99d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringInitCapDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringInitCapDescriptor.java
@@ -25,11 +25,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
@@ -49,7 +49,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractUnaryStringStringEval(ctx, args[0], StringInitCapDescriptor.this.getIdentifier()) {
@Override
protected void process(UTF8StringPointable inputString, IPointable resultPointable)
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringJoinDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringJoinDescriptor.java
index 67f3fbd..9e69011 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringJoinDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringJoinDescriptor.java
@@ -22,18 +22,24 @@
import java.io.IOException;
import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMissing;
+import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.evaluators.common.AsterixListAccessor;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -53,91 +59,102 @@
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
-
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IScalarEvaluatorFactory listEvalFactory = args[0];
- private IScalarEvaluatorFactory sepEvalFactory = args[1];
- private IPointable inputArgList = new VoidPointable();
- private IPointable inputArgSep = new VoidPointable();
- private IScalarEvaluator evalList = listEvalFactory.createScalarEvaluator(ctx);
- private IScalarEvaluator evalSep = sepEvalFactory.createScalarEvaluator(ctx);
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private final AsterixListAccessor listAccessor = new AsterixListAccessor();
+ private final DataOutput out = resultStorage.getDataOutput();
+ private final IScalarEvaluatorFactory listEvalFactory = args[0];
+ private final IScalarEvaluatorFactory sepEvalFactory = args[1];
+ private final IPointable inputArgList = new VoidPointable();
+ private final IPointable inputArgSep = new VoidPointable();
+ private final IScalarEvaluator evalList = listEvalFactory.createScalarEvaluator(ctx);
+ private final IScalarEvaluator evalSep = sepEvalFactory.createScalarEvaluator(ctx);
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ANULL);
+ private ISerializerDeserializer<AMissing> missingSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AMISSING);
private final byte[] tempLengthArray = new byte[5];
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
- try {
- resultStorage.reset();
- evalList.evaluate(tuple, inputArgList);
- byte[] serOrderedList = inputArgList.getByteArray();
- int serOrderedListOffset = inputArgList.getStartOffset();
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ evalList.evaluate(tuple, inputArgList);
+ evalSep.evaluate(tuple, inputArgSep);
- evalSep.evaluate(tuple, inputArgSep);
- byte[] serSep = inputArgSep.getByteArray();
- int serSepOffset = inputArgSep.getStartOffset();
-
- if (serOrderedList[serOrderedListOffset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
- && serOrderedList[serOrderedListOffset
- + 1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(AsterixBuiltinFunctions.STRING_JOIN.getName()
- + ": expects input type ORDEREDLIST but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER
- .deserialize(serOrderedList[serOrderedListOffset]));
- }
-
- if (serSep[serSepOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(AsterixBuiltinFunctions.STRING_JOIN.getName()
- + ": expects STRING type for the seperator but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serSep[serSepOffset]));
- }
-
- int size = AOrderedListSerializerDeserializer.getNumberOfItems(serOrderedList,
- serOrderedListOffset);
- try {
- // calculate length first
- int utf_8_len = 0;
- int sep_len = UTF8StringUtil.getUTFLength(serSep, serSepOffset + 1);
- int sep_meta_len = UTF8StringUtil.getNumBytesToStoreLength(sep_len);
-
- for (int i = 0; i < size; i++) {
- int itemOffset = AOrderedListSerializerDeserializer.getItemOffset(serOrderedList,
- serOrderedListOffset, i);
-
- int currentSize = UTF8StringUtil.getUTFLength(serOrderedList, itemOffset);
- if (i != size - 1 && currentSize != 0) {
- utf_8_len += sep_len;
- }
- utf_8_len += currentSize;
- }
- out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
- int length = UTF8StringUtil.encodeUTF8Length(utf_8_len, tempLengthArray, 0);
- out.write(tempLengthArray, 0, length);
- for (int i = 0; i < size; i++) {
- int itemOffset = AOrderedListSerializerDeserializer.getItemOffset(serOrderedList,
- serOrderedListOffset, i);
- utf_8_len = UTF8StringUtil.getUTFLength(serOrderedList, itemOffset);
- out.write(serOrderedList,
- itemOffset + UTF8StringUtil.getNumBytesToStoreLength(utf_8_len), utf_8_len);
- if (i == size - 1 || utf_8_len == 0) {
- continue;
- }
- for (int j = 0; j < sep_len; j++) {
- out.writeByte(serSep[serSepOffset + 1 + sep_meta_len + j]);
- }
- }
- } catch (AsterixException ex) {
- throw new AlgebricksException(ex);
- }
- result.set(resultStorage);
- } catch (IOException e1) {
- throw new AlgebricksException(e1.getMessage());
+ byte[] listBytes = inputArgList.getByteArray();
+ int listOffset = inputArgList.getStartOffset();
+ if (listBytes[listOffset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
+ && listBytes[listOffset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, listBytes[listOffset],
+ ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
+ ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
}
+ byte[] sepBytes = inputArgSep.getByteArray();
+ int sepOffset = inputArgSep.getStartOffset();
+ if (sepBytes[sepOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, sepBytes[sepOffset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+ int sepLen = UTF8StringUtil.getUTFLength(sepBytes, sepOffset + 1);
+ int sepMetaLen = UTF8StringUtil.getNumBytesToStoreLength(sepLen);
+
+ listAccessor.reset(listBytes, listOffset);
+ try {
+ // calculate length first
+ int utf8Len = 0;
+ int size = listAccessor.size();
+ for (int i = 0; i < size; i++) {
+ int itemOffset = listAccessor.getItemOffset(i);
+ ATypeTag itemType = listAccessor.getItemType(itemOffset);
+ // Increase the offset by 1 if the give list has heterogeneous elements,
+ // since the item itself has a typetag.
+ if (listAccessor.itemsAreSelfDescribing()) {
+ itemOffset += 1;
+ }
+ if (itemType != ATypeTag.STRING) {
+ if (itemType == ATypeTag.NULL) {
+ nullSerde.serialize(ANull.NULL, out);
+ result.set(resultStorage);
+ return;
+ }
+ if (itemType == ATypeTag.MISSING) {
+ missingSerde.serialize(AMissing.MISSING, out);
+ result.set(resultStorage);
+ return;
+ }
+ throw new UnsupportedItemTypeException(getIdentifier(), itemType.serialize());
+ }
+ int currentSize = UTF8StringUtil.getUTFLength(listBytes, itemOffset);
+ if (i != size - 1 && currentSize != 0) {
+ utf8Len += sepLen;
+ }
+ utf8Len += currentSize;
+ }
+
+ out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ int cbytes = UTF8StringUtil.encodeUTF8Length(utf8Len, tempLengthArray, 0);
+ out.write(tempLengthArray, 0, cbytes);
+ for (int i = 0; i < listAccessor.size(); i++) {
+ int itemOffset = listAccessor.getItemOffset(i);
+ if (listAccessor.itemsAreSelfDescribing()) {
+ itemOffset += 1;
+ }
+ utf8Len = UTF8StringUtil.getUTFLength(listBytes, itemOffset);
+ out.write(listBytes, UTF8StringUtil.getNumBytesToStoreLength(utf8Len) + itemOffset,
+ utf8Len);
+ for (int j = 0; j < sepLen; j++) {
+ out.writeByte(sepBytes[sepOffset + 1 + sepMetaLen + j]);
+ }
+ }
+ } catch (IOException | AsterixException ex) {
+ throw new HyracksDataException(ex);
+ }
+ result.set(resultStorage);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrim2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrim2Descriptor.java
index bcb70e1..69761b7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrim2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrim2Descriptor.java
@@ -26,11 +26,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.StringTrimmer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
public class StringLTrim2Descriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -49,13 +49,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractBinaryStringStringEval(ctx, args[0], args[1],
StringLTrim2Descriptor.this.getIdentifier()) {
private StringTrimmer stringTrimmer = new StringTrimmer(resultBuilder, resultArray);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrimDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrimDescriptor.java
index a21b6b4..22ae3b8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrimDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrimDescriptor.java
@@ -26,11 +26,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.StringTrimmer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
@@ -50,13 +50,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractUnaryStringStringEval(ctx, args[0], StringLTrimDescriptor.this.getIdentifier()) {
private StringTrimmer stringTrimmer = new StringTrimmer(resultBuilder, resultArray, " ");
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
index f1a0a4f..5c73a26 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
@@ -24,19 +24,19 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -58,7 +58,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private AMutableInt64 result = new AMutableInt64(0);
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -71,7 +71,7 @@
@Override
public void evaluate(IFrameTupleReference tuple, IPointable resultPointable)
- throws AlgebricksException {
+ throws HyracksDataException {
try {
resultStorage.reset();
eval.evaluate(tuple, inputArg);
@@ -83,13 +83,12 @@
result.setValue(len);
int64Serde.serialize(result, out);
} else {
- throw new AlgebricksException(AsterixBuiltinFunctions.STRING_LENGTH.getName()
- + ": expects input type STRING/NULL but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serString[offset]));
+ throw new TypeMismatchException(getIdentifier(), 0,
+ serString[offset], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
resultPointable.set(resultStorage);
} catch (IOException e1) {
- throw new AlgebricksException(e1);
+ throw new HyracksDataException(e1);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java
index 8adb7b3..e9bd020 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java
@@ -25,11 +25,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.RegExpMatcher;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
/**
@@ -54,13 +54,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractBinaryStringBoolEval(ctx, args[0], args[1],
StringLikeDescriptor.this.getIdentifier()) {
private final RegExpMatcher matcher = new RegExpMatcher();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
index bc27a11..6116018 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
@@ -24,11 +24,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
@@ -48,7 +48,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractUnaryStringStringEval(ctx, args[0], StringLowerCaseDescriptor.this.getIdentifier()) {
@Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
index 6194571..fbd747d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
@@ -25,11 +25,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
public class StringPositionDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -43,13 +43,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractBinaryStringIntEval(ctx, args[0], args[1],
StringPositionDescriptor.this.getIdentifier()) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrim2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrim2Descriptor.java
index bbd4822..dc89c40 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrim2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrim2Descriptor.java
@@ -26,11 +26,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.StringTrimmer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
public class StringRTrim2Descriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -49,13 +49,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractBinaryStringStringEval(ctx, args[0], args[1],
StringRTrim2Descriptor.this.getIdentifier()) {
private StringTrimmer stringTrimmer = new StringTrimmer(resultBuilder, resultArray);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrimDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrimDescriptor.java
index 89071d5..2738227 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrimDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrimDescriptor.java
@@ -26,11 +26,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.StringTrimmer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
@@ -50,13 +50,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractUnaryStringStringEval(ctx, args[0], StringRTrimDescriptor.this.getIdentifier()) {
private StringTrimmer stringTrimmer = new StringTrimmer(resultBuilder, resultArray, " ");
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsDescriptor.java
index e7017ab..290e76d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsDescriptor.java
@@ -25,11 +25,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.RegExpMatcher;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
public class StringRegExpContainsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -43,13 +43,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractBinaryStringBoolEval(ctx, args[0], args[1],
StringRegExpContainsDescriptor.this.getIdentifier()) {
private final RegExpMatcher matcher = new RegExpMatcher();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsWithFlagDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsWithFlagDescriptor.java
index 45f72b3..633ca17 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsWithFlagDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsWithFlagDescriptor.java
@@ -29,11 +29,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.RegExpMatcher;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
public class StringRegExpContainsWithFlagDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -47,13 +47,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractTripleStringBoolEval(ctx, args[0], args[1], args[2],
StringRegExpContainsWithFlagDescriptor.this.getIdentifier()) {
private final RegExpMatcher matcher = new RegExpMatcher();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeDescriptor.java
index 16e70b9..53c9dea 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeDescriptor.java
@@ -26,11 +26,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.RegExpMatcher;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
public class StringRegExpLikeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -44,13 +44,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractBinaryStringBoolEval(ctx, args[0], args[1],
StringRegExpLikeDescriptor.this.getIdentifier()) {
private final RegExpMatcher matcher = new RegExpMatcher();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeWithFlagDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeWithFlagDescriptor.java
index 274b36a..666f48b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeWithFlagDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeWithFlagDescriptor.java
@@ -26,11 +26,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.RegExpMatcher;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
public class StringRegExpLikeWithFlagDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -44,13 +44,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractTripleStringBoolEval(ctx, args[0], args[1], args[2],
StringRegExpLikeWithFlagDescriptor.this.getIdentifier()) {
private final RegExpMatcher matcher = new RegExpMatcher();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
index 3fd8544..41cf079 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
@@ -26,11 +26,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.RegExpMatcher;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
public class StringRegExpPositionDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -44,13 +44,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractBinaryStringIntEval(ctx, args[0], args[1],
StringRegExpPositionDescriptor.this.getIdentifier()) {
private final RegExpMatcher matcher = new RegExpMatcher();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
index 1085d89..64c2072 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
@@ -26,11 +26,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.RegExpMatcher;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
public class StringRegExpPositionWithFlagDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -44,13 +44,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractTripleStringIntEval(ctx, args[0], args[1], args[2],
StringRegExpPositionWithFlagDescriptor.this.getIdentifier()) {
private final RegExpMatcher matcher = new RegExpMatcher();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceDescriptor.java
index bdaf7cf..952bd7f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceDescriptor.java
@@ -25,11 +25,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.RegExpMatcher;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
public class StringRegExpReplaceDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -43,13 +43,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractTripleStringStringEval(ctx, args[0], args[1], args[2],
StringRegExpReplaceDescriptor.this.getIdentifier()) {
private final RegExpMatcher matcher = new RegExpMatcher();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceWithFlagsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceWithFlagsDescriptor.java
index cf6eeca..270bb7b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceWithFlagsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceWithFlagsDescriptor.java
@@ -25,11 +25,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.RegExpMatcher;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
public class StringRegExpReplaceWithFlagsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -43,13 +43,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractQuadStringStringEval(ctx, args[0], args[1], args[2], args[3],
StringRegExpReplaceWithFlagsDescriptor.this.getIdentifier()) {
private final RegExpMatcher matcher = new RegExpMatcher();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRepeatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRepeatDescriptor.java
index 0f4c0de..6b02783 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRepeatDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRepeatDescriptor.java
@@ -22,14 +22,15 @@
import java.io.DataOutput;
import java.io.IOException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -51,13 +52,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
// Argument evaluators.
private IScalarEvaluator evalString = args[0].createScalarEvaluator(ctx);
@@ -73,26 +73,22 @@
private byte[] tempLengthArray = new byte[5];
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
// Calls argument evaluators.
- evalStart.evaluate(tuple, argNumber);
evalString.evaluate(tuple, argString);
+ evalStart.evaluate(tuple, argNumber);
// Gets the repeating times.
- int repeatingTimes = 0;
byte[] bytes = argNumber.getByteArray();
int offset = argNumber.getStartOffset();
- try {
- repeatingTimes = ATypeHierarchy.getIntegerValue(bytes, offset);
- } catch (HyracksDataException e1) {
- throw new AlgebricksException(e1);
- }
+ int repeatingTimes = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 1, bytes,
+ offset);
// Checks repeatingTimes. It should be a non-negative value.
if (repeatingTimes < 0) {
- throw new AlgebricksException(StringRepeatDescriptor.this.getIdentifier().getName()
- + ": expects a non-negative repeating number but got " + repeatingTimes + ".");
+ throw new RuntimeDataException(ErrorCode.ERROR_NEGATIVE_VALUE, getIdentifier(), 1,
+ repeatingTimes);
}
// Gets the input string.
@@ -100,9 +96,8 @@
offset = argString.getStartOffset();
// Checks the type of the string argument.
if (bytes[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(StringRepeatDescriptor.this.getIdentifier().getName()
- + ": expects type STRING for the first argument but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]));
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
// Calculates the result string length.
@@ -119,7 +114,7 @@
out.write(bytes, inputStringStart, inputLen);
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringSplitDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringSplitDescriptor.java
index 8cd3a5b..894d912 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringSplitDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringSplitDescriptor.java
@@ -23,19 +23,19 @@
import java.io.IOException;
import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -60,7 +60,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
// Argument evaluators.
private final IScalarEvaluator stringEval = args[0].createScalarEvaluator(ctx);
@@ -84,7 +84,7 @@
private final DataOutput out = resultStorage.getDataOutput();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
// Calls argument evaluators.
@@ -97,9 +97,8 @@
int srcLen = argString.getLength();
// Type check for the first argument.
if (srcString[srcOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(StringSplitDescriptor.this.getIdentifier().getName()
- + ": expects input type STRING for the first argument but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(srcString[srcOffset]));
+ throw new TypeMismatchException(getIdentifier(), 0, srcString[srcOffset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
// Gets the bytes of the pattern string.
@@ -108,10 +107,8 @@
int patternLen = argPattern.getLength();
// Type check for the second argument.
if (patternString[patternOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(StringSplitDescriptor.this.getIdentifier().getName()
- + ": expects input type STRING for the second argument but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER
- .deserialize(patternString[patternOffset]));
+ throw new TypeMismatchException(getIdentifier(), 1, patternString[patternOffset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
// Sets the UTF8 String pointables.
@@ -144,7 +141,7 @@
listBuilder.write(out, true);
result.set(resultStorage);
} catch (IOException e1) {
- throw new AlgebricksException(e1);
+ throw new HyracksDataException(e1);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringStartsWithDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringStartsWithDescriptor.java
index 6a3910e..b4c16fc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringStartsWithDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringStartsWithDescriptor.java
@@ -24,11 +24,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
public class StringStartsWithDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -42,14 +42,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
-
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractBinaryStringBoolEval(ctx, args[0], args[1],
AsterixBuiltinFunctions.STRING_STARTS_WITH) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
index f0c2330..1409b65 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
@@ -25,20 +25,20 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -62,7 +62,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
protected final DataOutput out = resultStorage.getDataOutput();
@@ -79,7 +79,7 @@
private final AMutableInt64 aInt64 = new AMutableInt64(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
stringEval.evaluate(tuple, argPtr);
@@ -103,13 +103,12 @@
}
listBuilder.write(out, true);
} else {
- throw new AlgebricksException(AsterixBuiltinFunctions.STRING_TO_CODEPOINT.getName()
- + ": expects input type STRING but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serString[offset]));
+ throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
result.set(resultStorage);
} catch (IOException e1) {
- throw new AlgebricksException(e1.getMessage());
+ throw new HyracksDataException(e1);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrim2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrim2Descriptor.java
index d2b2d36..d9e30b0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrim2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrim2Descriptor.java
@@ -26,11 +26,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.StringTrimmer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
public class StringTrim2Descriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -49,13 +49,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractBinaryStringStringEval(ctx, args[0], args[1],
StringTrim2Descriptor.this.getIdentifier()) {
private StringTrimmer stringTrimmer = new StringTrimmer(resultBuilder, resultArray);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrimDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrimDescriptor.java
index 2e5ece0..6ddc343 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrimDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrimDescriptor.java
@@ -26,11 +26,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.StringTrimmer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
@@ -50,13 +50,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractUnaryStringStringEval(ctx, args[0], StringTrimDescriptor.this.getIdentifier()) {
private StringTrimmer stringTrimmer = new StringTrimmer(resultBuilder, resultArray, " ");
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java
index 9677f46..2bb16e7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java
@@ -24,11 +24,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
@@ -48,7 +48,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractUnaryStringStringEval(ctx, args[0], StringUpperCaseDescriptor.this.getIdentifier()) {
@Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
index f0c9fba..4c302a4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
@@ -21,14 +21,15 @@
import java.io.DataOutput;
import java.io.IOException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -52,15 +53,13 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
-
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
private IPointable argString = new VoidPointable();
@@ -72,44 +71,35 @@
private final UTF8StringPointable string = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
- evalStart.evaluate(tuple, argStart);
evalString.evaluate(tuple, argString);
+ evalStart.evaluate(tuple, argStart);
- int start = 0;
byte[] bytes = argStart.getByteArray();
int offset = argStart.getStartOffset();
- try {
- start = ATypeHierarchy.getIntegerValue(bytes, offset) - 1;
- } catch (HyracksDataException e1) {
- throw new AlgebricksException(e1);
- }
-
+ int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 1, bytes, offset) - 1;
bytes = argString.getByteArray();
offset = argString.getStartOffset();
int len = argString.getLength();
if (bytes[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(AsterixBuiltinFunctions.SUBSTRING2.getName()
- + ": expects type STRING for the first argument but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]));
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
string.set(bytes, offset + 1, len - 1);
array.reset();
try {
UTF8StringPointable.substr(string, start, Integer.MAX_VALUE, builder, array);
} catch (StringIndexOutOfBoundsException e) {
- throw new AlgebricksException(AsterixBuiltinFunctions.SUBSTRING.getName() + ": start="
- + start + "\tgoing past the input length.");
+ throw new RuntimeDataException(ErrorCode.ERROR_OUT_OF_BOUND, getIdentifier(), 1, start);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
-
try {
out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
out.write(array.getByteArray(), 0, array.getLength());
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java
index 23dbbec..b81524f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java
@@ -21,17 +21,17 @@
import java.io.DataOutput;
import java.io.IOException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -50,13 +50,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
@@ -70,7 +69,7 @@
private final UTF8StringPointable patternPtr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
evalString.evaluate(tuple, array0);
byte[] src = array0.getByteArray();
@@ -81,13 +80,15 @@
int patternOffset = array1.getStartOffset();
int patternLen = array1.getLength();
- if (src[srcOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG
- || pattern[patternOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(AsterixBuiltinFunctions.SUBSTRING_AFTER.getName()
- + ": expects input type (STRING/NULL, STRING/NULL) but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(src[srcOffset]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(pattern[patternOffset]) + ").");
+ if (src[srcOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, src[srcOffset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
+ if (pattern[patternOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, pattern[patternOffset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+
try {
stringPtr.set(src, srcOffset + 1, srcLen - 1);
patternPtr.set(pattern, patternOffset + 1, patternLen - 1);
@@ -97,7 +98,7 @@
out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
out.write(array.getByteArray(), 0, array.getLength());
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java
index 65aa34b..031de08 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java
@@ -21,17 +21,17 @@
import java.io.DataOutput;
import java.io.IOException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -50,13 +50,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -71,7 +70,7 @@
private final UTF8StringPointable patternPtr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
evalString.evaluate(tuple, array0);
byte[] src = array0.getByteArray();
@@ -83,12 +82,13 @@
int patternOffset = array1.getStartOffset();
int patternLen = array1.getLength();
- if (src[srcOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG
- || pattern[patternOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(AsterixBuiltinFunctions.SUBSTRING_BEFORE.getName()
- + ": expects input type (STRING/NULL, STRING/NULL) but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(src[srcOffset]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(pattern[patternOffset]) + ").");
+ if (src[srcOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, src[srcOffset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+ if (pattern[patternOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, pattern[patternOffset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
try {
@@ -99,7 +99,7 @@
out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
out.write(array.getByteArray(), 0, array.getLength());
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
index e960b58..65e6dab 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
@@ -21,13 +21,15 @@
import java.io.DataOutput;
import java.io.IOException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -53,13 +55,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -76,49 +77,43 @@
private final UTF8StringPointable string = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
evalString.evaluate(tuple, argString);
evalStart.evaluate(tuple, argStart);
evalLen.evaluate(tuple, argLen);
- int start = 0;
byte[] bytes = argStart.getByteArray();
int offset = argStart.getStartOffset();
- try {
- start = ATypeHierarchy.getIntegerValue(bytes, offset) - 1;
- } catch (HyracksDataException e1) {
- throw new AlgebricksException(e1);
- }
+ int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes, offset) - 1;
- int len = 0;
bytes = argLen.getByteArray();
offset = argLen.getStartOffset();
- try {
- len = ATypeHierarchy.getIntegerValue(bytes, offset);
- } catch (HyracksDataException e1) {
- throw new AlgebricksException(e1);
- }
+ int len = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 1, bytes, offset);
bytes = argString.getByteArray();
offset = argString.getStartOffset();
int length = argString.getLength();
+ if (bytes[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
string.set(bytes, offset + 1, length - 1);
array.reset();
try {
UTF8StringPointable.substr(string, start, len, builder, array);
} catch (StringIndexOutOfBoundsException e) {
- throw new AlgebricksException(AsterixBuiltinFunctions.SUBSTRING.getName() + ": start="
- + start + "\tgoing past the input length.");
+ throw new RuntimeDataException(ErrorCode.ERROR_OUT_OF_BOUND, getIdentifier(), 1,
+ start + len - 1);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
try {
out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
out.write(array.getByteArray(), 0, array.getLength());
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SwitchCaseDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SwitchCaseDescriptor.java
index 9e14bff..be01a03 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SwitchCaseDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SwitchCaseDescriptor.java
@@ -22,11 +22,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -47,14 +47,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
-
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
final IPointable condPtr = new VoidPointable();
final IPointable casePtr = new VoidPointable();
final IPointable argPtr = new VoidPointable();
@@ -76,7 +74,7 @@
return new IScalarEvaluator() {
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
int n = args.length;
evals[0].evaluate(tuple, condPtr);
for (int i = 1; i < n; i += 2) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/UUIDDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/UUIDDescriptor.java
index e306cc4..2827288 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/UUIDDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/UUIDDescriptor.java
@@ -29,7 +29,6 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -52,7 +51,7 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@@ -62,22 +61,18 @@
.getSerializerDeserializer(BuiltinType.AUUID);
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput output = resultStorage.getDataOutput();
private final AGeneratedUUID uuid = new AGeneratedUUID();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
uuid.nextUUID();
- try {
- resultStorage.reset();
- uuidSerDe.serialize(uuid, output);
- result.set(resultStorage);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ resultStorage.reset();
+ uuidSerDe.serialize(uuid, output);
+ result.set(resultStorage);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/UnorderedListConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/UnorderedListConstructorDescriptor.java
index ce08ba8..382d53e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/UnorderedListConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/UnorderedListConstructorDescriptor.java
@@ -28,11 +28,11 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnorderedListType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -84,7 +84,7 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
final IPointable inputVal = new VoidPointable();
final IScalarEvaluator[] argEvals = new IScalarEvaluator[args.length];
for (int i = 0; i < args.length; i++) {
@@ -97,7 +97,7 @@
private UnorderedListBuilder builder = new UnorderedListBuilder();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
builder.reset(unorderedlistType);
if (selfDescList) {
@@ -110,11 +110,11 @@
builder.write(out, true);
result.set(resultStorage);
} catch (IOException ioe) {
- throw new AlgebricksException(ioe);
+ throw new HyracksDataException(ioe);
}
}
- private void writeUntypedItems(IFrameTupleReference tuple) throws AlgebricksException {
+ private void writeUntypedItems(IFrameTupleReference tuple) throws HyracksDataException {
try {
for (int i = 0; i < argEvals.length; i++) {
@@ -122,11 +122,11 @@
builder.addItem(inputVal);
}
} catch (IOException ioe) {
- throw new AlgebricksException(ioe);
+ throw new HyracksDataException(ioe);
}
}
- private void writeTypedItems(IFrameTupleReference tuple) throws AlgebricksException {
+ private void writeTypedItems(IFrameTupleReference tuple) throws HyracksDataException {
try {
for (int i = 0; i < argEvals.length; i++) {
@@ -134,7 +134,7 @@
builder.addItem(inputVal);
}
} catch (IOException ioe) {
- throw new AlgebricksException(ioe);
+ throw new HyracksDataException(ioe);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/WordTokensDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/WordTokensDescriptor.java
index 7c1117f..3607083 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/WordTokensDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/WordTokensDescriptor.java
@@ -24,11 +24,11 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.WordTokensEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
@@ -50,13 +50,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
ITokenFactory tokenFactory = new UTF8WordTokenFactory();
IBinaryTokenizer tokenizer = new DelimitedUTF8StringBinaryTokenizer(true, true, tokenFactory);
return new WordTokensEvaluator(args, ctx, tokenizer, BuiltinType.ASTRING);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractBinaryScalarEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractBinaryScalarEvaluator.java
index c12df6f..bcd8094 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractBinaryScalarEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractBinaryScalarEvaluator.java
@@ -21,12 +21,13 @@
import java.io.DataOutput;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -39,7 +40,7 @@
protected IScalarEvaluator[] evaluators;
public AbstractBinaryScalarEvaluator(final IHyracksTaskContext context,
- final IScalarEvaluatorFactory[] evaluatorFactories) throws AlgebricksException {
+ final IScalarEvaluatorFactory[] evaluatorFactories) throws HyracksDataException {
pointables = new IPointable[evaluatorFactories.length];
evaluators = new IScalarEvaluator[evaluatorFactories.length];
for (int i = 0; i < evaluators.length; ++i) {
@@ -71,13 +72,12 @@
}
protected void checkTypeMachingThrowsIfNot(String title, ATypeTag[] expected, ATypeTag... actual)
- throws AlgebricksException {
+ throws HyracksDataException {
for (int i = 0; i < expected.length; i++) {
if (expected[i] != actual[i]) {
if (!ATypeHierarchy.canPromote(actual[i], expected[i])
&& !ATypeHierarchy.canPromote(expected[i], actual[i])) {
- throw new AlgebricksException(title + ": expects " + expected[i] + " at " + rankToString(i + 1)
- + " argument, but got " + actual[i]);
+ throw new TypeMismatchException(title, i, actual[i].serialize(), expected[i].serialize());
}
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
index b9acc4d..f0b78bf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
@@ -24,7 +24,6 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -46,41 +45,33 @@
.getSerializerDeserializer(BuiltinType.AINT64);
public AbstractFindBinaryEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory[] copyEvaluatorFactories,
- String functionName) throws AlgebricksException {
+ String functionName) throws HyracksDataException {
super(context, copyEvaluatorFactories);
this.functionName = functionName;
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable resultPointable) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable resultPointable) throws HyracksDataException {
resultStorage.reset();
for (int i = 0; i < pointables.length; ++i) {
evaluators[i].evaluate(tuple, pointables[i]);
}
int fromOffset = getFromOffset(tuple);
- try {
- ATypeTag textTag = ATypeTag.VALUE_TYPE_MAPPING[pointables[0].getByteArray()[pointables[0]
- .getStartOffset()]];
- ATypeTag wordTag = ATypeTag.VALUE_TYPE_MAPPING[pointables[1].getByteArray()[pointables[1]
- .getStartOffset()]];
+ ATypeTag textTag = ATypeTag.VALUE_TYPE_MAPPING[pointables[0].getByteArray()[pointables[0].getStartOffset()]];
+ ATypeTag wordTag = ATypeTag.VALUE_TYPE_MAPPING[pointables[1].getByteArray()[pointables[1].getStartOffset()]];
- checkTypeMachingThrowsIfNot(functionName, EXPECTED_INPUT_TAG, textTag, wordTag);
- textPtr.set(pointables[0].getByteArray(), pointables[0].getStartOffset() + 1,
- pointables[0].getLength() - 1);
- wordPtr.set(pointables[1].getByteArray(), pointables[0].getStartOffset() + 1,
- pointables[1].getLength() - 1);
- result.setValue(1L + indexOf(textPtr.getByteArray(), textPtr.getContentStartOffset(),
- textPtr.getContentLength(), wordPtr.getByteArray(), wordPtr.getContentStartOffset(),
- wordPtr.getContentLength(), fromOffset));
- intSerde.serialize(result, dataOutput);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ checkTypeMachingThrowsIfNot(functionName, EXPECTED_INPUT_TAG, textTag, wordTag);
+ textPtr.set(pointables[0].getByteArray(), pointables[0].getStartOffset() + 1, pointables[0].getLength() - 1);
+ wordPtr.set(pointables[1].getByteArray(), pointables[0].getStartOffset() + 1, pointables[1].getLength() - 1);
+ result.setValue(1L + indexOf(textPtr.getByteArray(), textPtr.getContentStartOffset(),
+ textPtr.getContentLength(), wordPtr.getByteArray(), wordPtr.getContentStartOffset(),
+ wordPtr.getContentLength(), fromOffset));
+ intSerde.serialize(result, dataOutput);
resultPointable.set(resultStorage);
}
- protected abstract int getFromOffset(IFrameTupleReference tuple) throws AlgebricksException;
+ protected abstract int getFromOffset(IFrameTupleReference tuple) throws HyracksDataException;
// copy from String.indexOf(String)
private int indexOf(byte[] source, int sourceOffset, int sourceCount, byte[] target, int targetOffset,
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
index 8b5c512..2834b67 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
@@ -21,11 +21,12 @@
import java.io.IOException;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -40,13 +41,13 @@
private static final ATypeTag[] EXPECTED_INPUT_TAGS = { ATypeTag.BINARY, ATypeTag.INT32 };
public AbstractSubBinaryEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory[] copyEvaluatorFactories,
- String functionName) throws AlgebricksException {
+ String functionName) throws HyracksDataException {
super(context, copyEvaluatorFactories);
this.functionName = functionName;
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
for (int i = 0; i < pointables.length; ++i) {
evaluators[i].evaluate(tuple, pointables[i]);
@@ -67,7 +68,8 @@
int subStart;
// strange SQL index convention
- subStart = ATypeHierarchy.getIntegerValue(startBytes, offset) - 1;
+ subStart = ATypeHierarchy.getIntegerValue(AsterixBuiltinFunctions.SUBBINARY_FROM.getName(), 1, startBytes,
+ offset) - 1;
int totalLength = byteArrayPointable.getContentLength();
int subLength = getSubLength(tuple);
@@ -89,10 +91,10 @@
dataOutput.write(byteArrayPointable.getByteArray(), byteArrayPointable.getContentStartOffset() + subStart,
subLength);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
- protected abstract int getSubLength(IFrameTupleReference tuple) throws AlgebricksException;
+ protected abstract int getSubLength(IFrameTupleReference tuple) throws HyracksDataException;
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryConcatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryConcatDescriptor.java
index b06203b..1221dc9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryConcatDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryConcatDescriptor.java
@@ -23,6 +23,7 @@
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMissing;
import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
@@ -31,7 +32,8 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.AsterixListAccessor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -59,13 +61,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractBinaryScalarEvaluator(ctx, args) {
private final AsterixListAccessor listAccessor = new AsterixListAccessor();
@@ -73,33 +74,36 @@
@SuppressWarnings("unchecked")
private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.ANULL);
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<AMissing> missingSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AMISSING);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
evaluators[0].evaluate(tuple, pointables[0]);
- ATypeTag typeTag = ATypeTag.VALUE_TYPE_MAPPING[pointables[0].getByteArray()[pointables[0]
- .getStartOffset()]];
- if (typeTag != ATypeTag.UNORDEREDLIST && typeTag != ATypeTag.ORDEREDLIST) {
- throw new AlgebricksException(getIdentifier().getName()
- + ": expects input type ORDEREDLIST/UNORDEREDLIST, but got " + typeTag);
+
+ byte[] data = pointables[0].getByteArray();
+ int offset = pointables[0].getStartOffset();
+ byte typeTag = data[offset];
+ if (typeTag != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG
+ && typeTag != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, typeTag,
+ ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG,
+ ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
}
try {
- byte[] data = pointables[0].getByteArray();
- int offset = pointables[0].getStartOffset();
-
listAccessor.reset(data, offset);
int concatLength = 0;
for (int i = 0; i < listAccessor.size(); i++) {
int itemOffset = listAccessor.getItemOffset(i);
ATypeTag itemType = listAccessor.getItemType(itemOffset);
if (itemType != ATypeTag.BINARY) {
- if (serializeNullIfAnyNull(itemType)) {
+ if (serializeUnknownIfAnyUnknown(itemType)) {
result.set(resultStorage);
return;
}
- throw new AlgebricksException(getIdentifier().getName()
- + ": expects type STRING/NULL for the list item but got " + itemType);
+ throw new UnsupportedItemTypeException(getIdentifier(), itemType.serialize());
}
concatLength += ByteArrayPointable.getContentLength(data, itemOffset);
}
@@ -113,22 +117,24 @@
dataOutput.write(data,
itemOffset + ByteArrayPointable.getNumberBytesToStoreMeta(length), length);
}
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
} catch (AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
- private boolean serializeNullIfAnyNull(ATypeTag... tags) throws HyracksDataException {
+ private boolean serializeUnknownIfAnyUnknown(ATypeTag... tags) throws HyracksDataException {
for (ATypeTag typeTag : tags) {
if (typeTag == ATypeTag.NULL) {
nullSerde.serialize(ANull.NULL, dataOutput);
return true;
}
+ if (typeTag == ATypeTag.MISSING) {
+ missingSerde.serialize(AMissing.MISSING, dataOutput);
+ return true;
+ }
}
return false;
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryLengthDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryLengthDescriptor.java
index 0a50c24..8b0f660 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryLengthDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryLengthDescriptor.java
@@ -28,7 +28,6 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -51,13 +50,12 @@
private static final ATypeTag[] EXPECTED_TAGS = { ATypeTag.BINARY };
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractBinaryScalarEvaluator(ctx, args) {
private AMutableInt64 result = new AMutableInt64(0);
@@ -67,20 +65,16 @@
@Override
public void evaluate(IFrameTupleReference tuple, IPointable resultPointable)
- throws AlgebricksException {
+ throws HyracksDataException {
resultStorage.reset();
evaluators[0].evaluate(tuple, pointables[0]);
ATypeTag tag = ATypeTag.VALUE_TYPE_MAPPING[pointables[0].getByteArray()[pointables[0]
.getStartOffset()]];
- try {
- checkTypeMachingThrowsIfNot(getIdentifier().getName(), EXPECTED_TAGS, tag);
- int len = ByteArrayPointable.getContentLength(pointables[0].getByteArray(),
+ checkTypeMachingThrowsIfNot(getIdentifier().getName(), EXPECTED_TAGS, tag);
+ int len = ByteArrayPointable.getContentLength(pointables[0].getByteArray(),
pointables[0].getStartOffset() + 1);
- result.setValue(len);
- intSerde.serialize(result, dataOutput);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ result.setValue(len);
+ intSerde.serialize(result, dataOutput);
resultPointable.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
index 94ec311..b09c0d4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
public class FindBinaryDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -45,16 +45,15 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractFindBinaryEvaluator(ctx, args, getIdentifier().getName()) {
@Override
- protected int getFromOffset(IFrameTupleReference tuple) throws AlgebricksException {
+ protected int getFromOffset(IFrameTupleReference tuple) throws HyracksDataException {
return 0;
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
index 5ed3773..c6ed136 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
@@ -24,7 +24,6 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -48,24 +47,18 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractFindBinaryEvaluator(ctx, args, getIdentifier().getName()) {
@Override
- protected int getFromOffset(IFrameTupleReference tuple) throws AlgebricksException {
- int getFrom = 0;
- try {
- getFrom = ATypeHierarchy.getIntegerValue(pointables[2].getByteArray(),
+ protected int getFromOffset(IFrameTupleReference tuple) throws HyracksDataException {
+ return ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 2,
+ pointables[2].getByteArray(),
pointables[2].getStartOffset());
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
- return getFrom;
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/ParseBinaryDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/ParseBinaryDescriptor.java
index 2488710..dfceccb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/ParseBinaryDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/ParseBinaryDescriptor.java
@@ -28,7 +28,7 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -61,13 +61,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractBinaryScalarEvaluator(ctx, args) {
@SuppressWarnings("unchecked")
@@ -82,41 +81,36 @@
private final Base64Parser base64Parser = new Base64Parser();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
evaluators[0].evaluate(tuple, pointables[0]);
evaluators[1].evaluate(tuple, pointables[1]);
- try {
- ATypeTag binaryTag = ATypeTag.VALUE_TYPE_MAPPING[pointables[0].getByteArray()[pointables[0]
- .getStartOffset()]];
+ ATypeTag binaryTag = ATypeTag.VALUE_TYPE_MAPPING[pointables[0].getByteArray()[pointables[0]
+ .getStartOffset()]];
- ATypeTag formatTag = ATypeTag.VALUE_TYPE_MAPPING[pointables[1].getByteArray()[pointables[1]
- .getStartOffset()]];
- checkTypeMachingThrowsIfNot(getIdentifier().getName(), EXPECTED_INPUT_TAGS, binaryTag,
- formatTag);
- stringPointable.set(pointables[0].getByteArray(), pointables[0].getStartOffset() + 1,
- pointables[0].getLength());
- formatPointable.set(pointables[1].getByteArray(), pointables[1].getStartOffset() + 1,
- pointables[1].getLength());
- if (HEX_FORMAT.ignoreCaseCompareTo(formatPointable) == 0) {
- hexParser.generateByteArrayFromHexString(stringPointable.getByteArray(),
- stringPointable.getCharStartOffset(), stringPointable.getUTF8Length());
+ ATypeTag formatTag = ATypeTag.VALUE_TYPE_MAPPING[pointables[1].getByteArray()[pointables[1]
+ .getStartOffset()]];
+ checkTypeMachingThrowsIfNot(getIdentifier().getName(), EXPECTED_INPUT_TAGS, binaryTag,
+ formatTag);
+ stringPointable.set(pointables[0].getByteArray(), pointables[0].getStartOffset() + 1,
+ pointables[0].getLength());
+ formatPointable.set(pointables[1].getByteArray(), pointables[1].getStartOffset() + 1,
+ pointables[1].getLength());
+ if (HEX_FORMAT.ignoreCaseCompareTo(formatPointable) == 0) {
+ hexParser.generateByteArrayFromHexString(stringPointable.getByteArray(),
+ stringPointable.getCharStartOffset(), stringPointable.getUTF8Length());
- aBinary.setValue(hexParser.getByteArray(), 0, hexParser.getLength());
- } else if (BASE64_FORMAT.ignoreCaseCompareTo(formatPointable) == 0) {
- base64Parser.generatePureByteArrayFromBase64String(stringPointable.getByteArray(),
- stringPointable.getCharStartOffset(), stringPointable.getUTF8Length());
+ aBinary.setValue(hexParser.getByteArray(), 0, hexParser.getLength());
+ } else if (BASE64_FORMAT.ignoreCaseCompareTo(formatPointable) == 0) {
+ base64Parser.generatePureByteArrayFromBase64String(stringPointable.getByteArray(),
+ stringPointable.getCharStartOffset(), stringPointable.getUTF8Length());
- aBinary.setValue(base64Parser.getByteArray(), 0, base64Parser.getLength());
- } else {
- throw new AlgebricksException(getIdentifier().getName()
- + ": expects format indicator of \"hex\" or \"base64\" in the 2nd argument");
- }
- binarySerde.serialize(aBinary, dataOutput);
- } catch (HyracksDataException e) {
- e.printStackTrace();
+ aBinary.setValue(base64Parser.getByteArray(), 0, base64Parser.getLength());
+ } else {
+ throw new UnsupportedItemTypeException(getIdentifier(), formatTag.serialize());
}
+ binarySerde.serialize(aBinary, dataOutput);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/PrintBinaryDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/PrintBinaryDescriptor.java
index 2dc1e34..3472627 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/PrintBinaryDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/PrintBinaryDescriptor.java
@@ -26,7 +26,7 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -60,13 +60,12 @@
public final static ATypeTag[] EXPECTED_INPUT_TAGS = { ATypeTag.BINARY, ATypeTag.STRING };
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractBinaryScalarEvaluator(ctx, args) {
private StringBuilder stringBuilder = new StringBuilder();
@@ -75,7 +74,7 @@
private final UTF8StringWriter writer = new UTF8StringWriter();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
evaluators[0].evaluate(tuple, pointables[0]);
evaluators[1].evaluate(tuple, pointables[1]);
@@ -102,15 +101,12 @@
Base64Printer.printBase64Binary(byteArrayPtr.getByteArray(),
byteArrayPtr.getContentStartOffset(), lengthBinary, stringBuilder);
} else {
- throw new AlgebricksException(getIdentifier().getName()
- + ": expects format indicator of \"hex\" or \"base64\" in the 2nd argument");
+ throw new UnsupportedItemTypeException(getIdentifier(), arg1Tag.serialize());
}
dataOutput.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
writer.writeUTF8(stringBuilder.toString(), dataOutput);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
index 129cd54..bd0562a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
@@ -23,11 +23,11 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
public class SubBinaryFromDescriptor extends AbstractScalarFunctionDynamicDescriptor {
@@ -45,16 +45,15 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractSubBinaryEvaluator(ctx, args, getIdentifier().getName()) {
@Override
- protected int getSubLength(IFrameTupleReference tuple) throws AlgebricksException {
+ protected int getSubLength(IFrameTupleReference tuple) throws HyracksDataException {
return Integer.MAX_VALUE;
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
index bb23877..a34f3cf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
@@ -24,7 +24,6 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -47,26 +46,18 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractSubBinaryEvaluator(ctx, args, getIdentifier().getName()) {
@Override
- protected int getSubLength(IFrameTupleReference tuple) throws AlgebricksException {
- int subLength = 0;
- try {
- subLength = ATypeHierarchy.getIntegerValue(pointables[2].getByteArray(),
- pointables[2].getStartOffset());
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
-
- return subLength;
+ protected int getSubLength(IFrameTupleReference tuple) throws HyracksDataException {
+ return ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 2,
+ pointables[2].getByteArray(), pointables[2].getStartOffset());
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java
index bc3ec40..134b55c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java
@@ -23,17 +23,18 @@
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.util.NonTaggedFormatUtil;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.IntegerPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -58,7 +59,7 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
@@ -80,7 +81,7 @@
* This method outputs into IHyracksTaskContext context [field type tag (1 byte)][the field data]
*/
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval0.evaluate(tuple, inputArg0);
@@ -88,16 +89,15 @@
int offset = inputArg0.getStartOffset();
if (serRecord[offset] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
- throw new AlgebricksException("Field accessor is not defined for values of type "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serRecord[offset]));
+ throw new TypeMismatchException(AsterixBuiltinFunctions.FIELD_ACCESS_BY_INDEX, 0,
+ serRecord[offset], ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
}
eval1.evaluate(tuple, inputArg1);
byte[] indexBytes = inputArg1.getByteArray();
int indexOffset = inputArg1.getStartOffset();
if (indexBytes[indexOffset] != ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
- throw new AlgebricksException("Field accessor is not defined for "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(indexBytes[indexOffset])
- + " as the second argument.");
+ throw new TypeMismatchException(AsterixBuiltinFunctions.FIELD_ACCESS_BY_INDEX, 1,
+ indexBytes[offset], ATypeTag.SERIALIZED_INT32_TYPE_TAG);
}
fieldIndex = IntegerPointable.getInteger(indexBytes, indexOffset + 1);
fieldValueType = recordType.getFieldTypes()[fieldIndex];
@@ -136,9 +136,9 @@
out.write(serRecord, fieldValueOffset, fieldValueLength);
result.set(resultStorage);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
} catch (AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
index 62779f3..bad37d0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
@@ -23,14 +23,15 @@
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.util.NonTaggedFormatUtil;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -50,7 +51,7 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -65,7 +66,7 @@
private ATypeTag fieldValueTypeTag;
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval0.evaluate(tuple, inputArg0);
@@ -75,11 +76,9 @@
int serRecordLen = inputArg0.getLength();
if (serRecord[serRecordOffset] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
- throw new AlgebricksException(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME.getName()
- + ": expects input type NULL or RECORD, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serRecord[serRecordOffset]));
+ throw new TypeMismatchException(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME, 0,
+ serRecord[serRecordOffset], ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
}
-
byte[] serFldName = inputArg1.getByteArray();
int serFldNameOffset = inputArg1.getStartOffset();
fieldValueOffset = ARecordSerializerDeserializer.getFieldOffsetByName(serRecord, serRecordOffset,
@@ -95,9 +94,9 @@
fieldValueTypeTag, true) + 1;
result.set(serRecord, fieldValueOffset, fieldValueLength);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
} catch (AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
index 7b53ed1..e366dce 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
@@ -28,6 +28,7 @@
import org.apache.asterix.om.base.AMissing;
import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
@@ -36,7 +37,8 @@
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.types.runtime.RuntimeRecordTypeInfo;
import org.apache.asterix.om.util.NonTaggedFormatUtil;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -65,7 +67,7 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -92,24 +94,20 @@
}
@SuppressWarnings("unchecked")
- private void generateFieldsPointables() throws AlgebricksException {
+ private void generateFieldsPointables() throws HyracksDataException {
for (int i = 0; i < fieldPath.size(); i++) {
ArrayBackedValueStorage storage = new ArrayBackedValueStorage();
DataOutput out = storage.getDataOutput();
AString as = new AString(fieldPath.get(i));
- try {
- AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(as.getType()).serialize(as,
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(as.getType()).serialize(as,
out);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
fieldPointables[i] = new VoidPointable();
fieldPointables[i].set(storage);
}
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
eval0.evaluate(tuple, inputArg0);
@@ -119,8 +117,8 @@
int len = inputArg0.getLength();
if (serRecord[start] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
- throw new AlgebricksException("Field accessor is not defined for values of type "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serRecord[start]));
+ throw new TypeMismatchException(AsterixBuiltinFunctions.FIELD_ACCESS_NESTED, 0,
+ serRecord[start], ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
}
int subFieldIndex = -1;
@@ -140,9 +138,11 @@
if (subType.getTypeTag().equals(ATypeTag.UNION)) {
//enforced SubType
subType = ((AUnionType) subType).getActualType();
- if (subType.getTypeTag().serialize() != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
- throw new AlgebricksException(
- "Field accessor is not defined for values of type " + subTypeTag);
+ byte serializedTypeTag = subType.getTypeTag().serialize();
+ if (serializedTypeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
+ throw new UnsupportedTypeException(
+ AsterixBuiltinFunctions.FIELD_ACCESS_NESTED.getName(),
+ serializedTypeTag);
}
if (subType.getTypeTag() == ATypeTag.RECORD) {
recTypeInfos[pathIndex].reset((ARecordType) subType);
@@ -197,8 +197,8 @@
// type check
if (pathIndex < fieldPointables.length - 1
&& serRecord[start] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
- throw new AlgebricksException("Field accessor is not defined for values of type "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serRecord[start]));
+ throw new UnsupportedTypeException(AsterixBuiltinFunctions.FIELD_ACCESS_NESTED,
+ serRecord[start]);
}
}
@@ -231,8 +231,8 @@
return;
}
if (serRecord[start] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
- throw new AlgebricksException("Field accessor is not defined for values of type "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serRecord[start]));
+ throw new UnsupportedTypeException(
+ AsterixBuiltinFunctions.FIELD_ACCESS_NESTED.getName(), serRecord[start]);
}
}
// emit the final result.
@@ -243,10 +243,8 @@
out.write(serRecord, subFieldOffset, subFieldLength);
result.set(resultStorage);
}
- } catch (IOException e) {
- throw new AlgebricksException(e);
- } catch (AsterixException e) {
- throw new AlgebricksException(e);
+ } catch (IOException | AsterixException e) {
+ throw new HyracksDataException(e);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
index 80e5fa5..3bea83d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
@@ -23,15 +23,17 @@
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.runtime.RuntimeRecordTypeInfo;
import org.apache.asterix.om.util.NonTaggedFormatUtil;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -53,7 +55,7 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -70,7 +72,7 @@
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
resultStorage.reset();
fieldNameEval.evaluate(tuple, inputArg1);
@@ -84,8 +86,8 @@
int serRecordLen = inputArg0.getLength();
if (serRecord[serRecordOffset] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
- throw new AlgebricksException("Field accessor is not defined for values of type "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serRecord[serRecordOffset]));
+ throw new TypeMismatchException(AsterixBuiltinFunctions.GET_RECORD_FIELD_VALUE, 0,
+ serRecord[serRecordOffset], ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
}
int subFieldOffset = -1;
@@ -129,9 +131,9 @@
// write result.
result.set(serRecord, subFieldOffset, subFieldLength);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
} catch (AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
index 0471b74..6f66314 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
@@ -22,14 +22,15 @@
import java.io.IOException;
import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.pointables.nonvisitor.ARecordPointable;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -48,7 +49,7 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ARecordPointable recordPointable = (ARecordPointable) ARecordPointable.FACTORY
@@ -60,7 +61,7 @@
private RecordFieldsUtil rfu = new RecordFieldsUtil();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, inputArg0);
byte[] data = inputArg0.getByteArray();
@@ -68,17 +69,17 @@
int len = inputArg0.getLength();
if (data[offset] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
- throw new AlgebricksException("Field accessor is not defined for values of type "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[offset]));
+ throw new TypeMismatchException(AsterixBuiltinFunctions.GET_RECORD_FIELDS, 0, data[offset],
+ ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
}
recordPointable.set(data, offset, len);
try {
rfu.processRecord(recordPointable, recordType, out, 0);
} catch (IOException e) {
- e.printStackTrace();
+ throw new HyracksDataException(e);
} catch (AsterixException e) {
- e.printStackTrace();
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
index 6565935..d9f3e68 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
@@ -24,6 +24,8 @@
import org.apache.asterix.builders.RecordBuilder;
import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.dataflow.data.nontagged.comparators.ListItemBinaryComparatorFactory;
import org.apache.asterix.dataflow.data.nontagged.hash.ListItemBinaryHashFunctionFactory;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
@@ -38,13 +40,13 @@
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.types.runtime.RuntimeRecordTypeInfo;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.BinaryHashMap;
import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -82,14 +84,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
final PointableAllocator allocator = new PointableAllocator();
final IVisitablePointable vp0 = allocator.allocateRecordValue(inRecType);
final IVisitablePointable vp1 = allocator.allocateListValue(inListType);
@@ -107,7 +107,7 @@
pointableHelper.serializeString("field-name", fieldNamePointable, true);
pointableHelper.serializeString("field-value", fieldValuePointer, true);
} catch (AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
return new IScalarEvaluator() {
@@ -131,7 +131,7 @@
private DataOutput out = resultStorage.getDataOutput();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
recordBuilder.reset(outRecType);
requiredRecordTypeInfo.reset(outRecType);
@@ -139,46 +139,42 @@
eval1.evaluate(tuple, argPtr1);
// Make sure we get a valid record
- if (argPtr0.getByteArray()[argPtr0.getStartOffset()] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
- throw new AlgebricksException("Expected an ordederlist of type " + inRecType + " but "
- + "got " + EnumDeserializer.ATYPETAGDESERIALIZER
- .deserialize(argPtr0.getByteArray()[argPtr0.getStartOffset()]));
+ byte typeTag0 = argPtr0.getByteArray()[argPtr0.getStartOffset()];
+ if (typeTag0 != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, typeTag0,
+ ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
}
// Make sure we get a valid list
- if (argPtr1.getByteArray()[argPtr1
- .getStartOffset()] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
- throw new AlgebricksException("Expected an ordederlist of type " + inListType + " but "
- + "got " + EnumDeserializer.ATYPETAGDESERIALIZER
- .deserialize(argPtr1.getByteArray()[argPtr1.getStartOffset()]));
+ byte typeTag1 = argPtr1.getByteArray()[argPtr1.getStartOffset()];
+ if (typeTag1 != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, typeTag1,
+ ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
}
vp0.set(argPtr0);
vp1.set(argPtr1);
- try {
- ARecordVisitablePointable recordPointable = (ARecordVisitablePointable) vp0;
- AListVisitablePointable listPointable = (AListVisitablePointable) vp1;
- // Initialize our hashmap
- int tableSize = recordPointable.getFieldNames().size() + listPointable.getItems().size();
- // Construct a new hash table only if table size is larger than the default
- // Thus avoiding unnecessary object construction
- if (hashMap == null || tableSize > TABLE_SIZE) {
- hashMap = new BinaryHashMap(tableSize, TABLE_FRAME_SIZE, putHashFunc, getHashFunc, cmp);
- } else {
- hashMap.clear();
- }
- addFields(recordPointable, listPointable);
- recordBuilder.write(out, true);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
+ ARecordVisitablePointable recordPointable = (ARecordVisitablePointable) vp0;
+ AListVisitablePointable listPointable = (AListVisitablePointable) vp1;
+
+ // Initialize our hashmap
+ int tableSize = recordPointable.getFieldNames().size() + listPointable.getItems().size();
+ // Construct a new hash table only if table size is larger than the default
+ // Thus avoiding unnecessary object construction
+ if (hashMap == null || tableSize > TABLE_SIZE) {
+ hashMap = new BinaryHashMap(tableSize, TABLE_FRAME_SIZE, putHashFunc, getHashFunc, cmp);
+ } else {
+ hashMap.clear();
}
+ addFields(recordPointable, listPointable);
+ recordBuilder.write(out, true);
result.set(resultStorage);
}
private void addFields(ARecordVisitablePointable inputRecordPointer,
- AListVisitablePointable listPointable) throws AlgebricksException {
+ AListVisitablePointable listPointable) throws HyracksDataException {
List<IVisitablePointable> inputRecordFieldNames = inputRecordPointer.getFieldNames();
List<IVisitablePointable> inputRecordFieldValues = inputRecordPointer.getFieldValues();
List<IVisitablePointable> inputFields = listPointable.getItems();
@@ -228,7 +224,7 @@
}
if (namePointable == null || valuePointable == null) {
- throw new AlgebricksException("Trying to add a null field name or field value");
+ throw new InvalidDataFormatException(getIdentifier(), "fields to be added");
}
// Check that the field being added is a valid field
@@ -243,7 +239,8 @@
tempValReference.set(entry.buf, entry.off, entry.len);
// If value is not equal throw conflicting duplicate field, otherwise ignore
if (!PointableHelper.byteArrayEqual(valuePointable, tempValReference)) {
- throw new AlgebricksException("Conflicting duplicate field found.");
+ throw new RuntimeDataException(ErrorCode.ERROR_DUPLICATE_FIELD,
+ getIdentifier());
}
} else {
if (pos > -1) {
@@ -256,8 +253,8 @@
hashMap.put(keyEntry, valEntry);
}
}
- } catch (AsterixException | HyracksDataException e) {
- throw new AlgebricksException(e);
+ } catch (AsterixException e) {
+ throw new HyracksDataException(e);
}
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
index 93c1f1b..b99b488 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
@@ -44,7 +44,6 @@
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.util.container.IObjectPool;
import org.apache.asterix.om.util.container.ListObjectPool;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IMutableValueStorage;
@@ -83,7 +82,7 @@
private final static ARecordType openType = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
public void processRecord(ARecordPointable recordAccessor, ARecordType recType, DataOutput out, int level)
- throws IOException, AsterixException, AlgebricksException {
+ throws IOException, AsterixException {
if (level == 0) {
// Resets pools for recycling objects before processing a top-level record.
resetPools();
@@ -221,7 +220,7 @@
}
public void addListField(IValueReference listArg, IAType fieldType, IARecordBuilder fieldRecordBuilder, int level)
- throws AsterixException, IOException, AlgebricksException {
+ throws AsterixException, IOException {
ArrayBackedValueStorage fieldAbvs = getTempBuffer();
ArrayBackedValueStorage valueAbvs = getTempBuffer();
@@ -235,7 +234,7 @@
}
public void addNestedField(IValueReference recordArg, IAType fieldType, IARecordBuilder fieldRecordBuilder,
- int level) throws HyracksDataException, AlgebricksException, IOException, AsterixException {
+ int level) throws IOException, AsterixException {
ArrayBackedValueStorage fieldAbvs = getTempBuffer();
ArrayBackedValueStorage valueAbvs = getTempBuffer();
@@ -257,7 +256,7 @@
}
public void processListValue(IValueReference listArg, IAType fieldType, DataOutput out, int level)
- throws AsterixException, IOException, AlgebricksException {
+ throws AsterixException, IOException {
ArrayBackedValueStorage itemValue = getTempBuffer();
IARecordBuilder listRecordBuilder = getRecordBuilder();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
index 2f54c9e..8e0d7ce 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
@@ -25,6 +25,8 @@
import org.apache.asterix.builders.RecordBuilder;
import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -40,11 +42,11 @@
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.comparisons.DeepEqualAssessor;
import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -79,14 +81,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
final PointableAllocator pa = new PointableAllocator();
final IVisitablePointable vp0 = pa.allocateRecordValue(inRecType0);
final IVisitablePointable vp1 = pa.allocateRecordValue(inRecType1);
@@ -109,7 +109,7 @@
private DataOutput out = resultStorage.getDataOutput();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -124,14 +124,14 @@
mergeFields(outRecType, rp0, rp1, true, 0);
rbStack.get(0).write(out, true);
} catch (IOException | AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
private void mergeFields(ARecordType combinedType, ARecordVisitablePointable leftRecord,
ARecordVisitablePointable rightRecord, boolean openFromParent, int nestedLevel)
- throws IOException, AsterixException, AlgebricksException {
+ throws IOException, AsterixException, HyracksDataException {
if (rbStack.size() < (nestedLevel + 1)) {
rbStack.add(new RecordBuilder());
}
@@ -160,7 +160,8 @@
openFromParent, nestedLevel);
foundMatch = true;
} else {
- throw new AlgebricksException("Duplicate field found");
+ throw new RuntimeDataException(ErrorCode.ERROR_DUPLICATE_FIELD,
+ getIdentifier());
}
}
}
@@ -197,7 +198,7 @@
*/
private void addFieldToSubRecord(ARecordType combinedType, IVisitablePointable fieldNamePointable,
IVisitablePointable leftValue, IVisitablePointable rightValue, boolean openFromParent,
- int nestedLevel) throws IOException, AsterixException, AlgebricksException {
+ int nestedLevel) throws IOException, AsterixException, HyracksDataException {
runtimeRecordTypeInfo.reset(combinedType);
int pos = runtimeRecordTypeInfo.getFieldIndex(fieldNamePointable.getByteArray(),
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
index b6aedb4..f8904df 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
@@ -26,6 +26,7 @@
import org.apache.asterix.builders.RecordBuilder;
import org.apache.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
import org.apache.asterix.om.base.AString;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -34,9 +35,7 @@
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -74,7 +73,7 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
// For writing each individual output record.
final ArrayBackedValueStorage itemStorage = new ArrayBackedValueStorage();
final DataOutput itemOutput = itemStorage.getDataOutput();
@@ -94,7 +93,7 @@
serde.serialize(new AString("name"), nameStorage.getDataOutput());
serde.serialize(new AString("value"), valueStorage.getDataOutput());
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
return new IScalarEvaluator() {
@@ -104,41 +103,36 @@
recType);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
- try {
- // Resets the result storage.
- resultStorage.reset();
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ // Resets the result storage.
+ resultStorage.reset();
- // Gets the input record.
- argEvaluator.evaluate(tuple, argPtr);
- byte inputTypeTag = argPtr.getByteArray()[argPtr.getStartOffset()];
- if (inputTypeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
- throw new AlgebricksException("Function " + RecordPairsDescriptor.this.getIdentifier()
- + " expects a record as the input, " + "but get a "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(inputTypeTag));
- }
- recordVisitablePointable.set(argPtr);
-
- listBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
- List<IVisitablePointable> fieldNames = recordVisitablePointable.getFieldNames();
- List<IVisitablePointable> fieldValues = recordVisitablePointable.getFieldValues();
- // Adds each field of the input record as a key-value pair into the result.
- int numFields = recordVisitablePointable.getFieldNames().size();
- for (int fieldIndex = 0; fieldIndex < numFields; ++fieldIndex) {
- itemStorage.reset();
- recBuilder.init();
- recBuilder.addField(nameStorage, fieldNames.get(fieldIndex));
- recBuilder.addField(valueStorage, fieldValues.get(fieldIndex));
- recBuilder.write(itemOutput, true);
- listBuilder.addItem(itemStorage);
- }
-
- // Writes the result and sets the result pointable.
- listBuilder.write(resultOut, true);
- result.set(resultStorage);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
+ // Gets the input record.
+ argEvaluator.evaluate(tuple, argPtr);
+ byte inputTypeTag = argPtr.getByteArray()[argPtr.getStartOffset()];
+ if (inputTypeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, inputTypeTag,
+ ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
}
+ recordVisitablePointable.set(argPtr);
+
+ listBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
+ List<IVisitablePointable> fieldNames = recordVisitablePointable.getFieldNames();
+ List<IVisitablePointable> fieldValues = recordVisitablePointable.getFieldValues();
+ // Adds each field of the input record as a key-value pair into the result.
+ int numFields = recordVisitablePointable.getFieldNames().size();
+ for (int fieldIndex = 0; fieldIndex < numFields; ++fieldIndex) {
+ itemStorage.reset();
+ recBuilder.init();
+ recBuilder.addField(nameStorage, fieldNames.get(fieldIndex));
+ recBuilder.addField(valueStorage, fieldValues.get(fieldIndex));
+ recBuilder.write(itemOutput, true);
+ listBuilder.addItem(itemStorage);
+ }
+
+ // Writes the result and sets the result pointable.
+ listBuilder.write(resultOut, true);
+ result.set(resultStorage);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
index b7e719d..593d684 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
@@ -25,7 +25,6 @@
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -52,8 +51,7 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new RecordRemoveFieldsEvalFactory(args[0], args[1], outputRecordType, inputRecType, inputListType);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsEvalFactory.java
index 68865c3..57b8c12 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsEvalFactory.java
@@ -28,6 +28,7 @@
import org.apache.asterix.builders.RecordBuilder;
import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.pointables.AListVisitablePointable;
import org.apache.asterix.om.pointables.ARecordVisitablePointable;
@@ -37,10 +38,8 @@
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.runtime.RuntimeRecordTypeInfo;
import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -70,7 +69,7 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
final PointableAllocator pa = new PointableAllocator();
final IVisitablePointable vp0 = pa.allocateRecordValue(inputRecType);
@@ -91,23 +90,21 @@
private DataOutput out = resultStorage.getDataOutput();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, inputArg0);
eval1.evaluate(tuple, inputArg1);
- if (inputArg0.getByteArray()[inputArg0.getStartOffset()] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
- throw new AlgebricksException(
- AsterixBuiltinFunctions.REMOVE_FIELDS.getName() + ": expects input type " + inputRecType
- + ", but got " + EnumDeserializer.ATYPETAGDESERIALIZER
- .deserialize(inputArg0.getByteArray()[inputArg0.getStartOffset()]));
+ byte inputTypeTag0 = inputArg0.getByteArray()[inputArg0.getStartOffset()];
+ if (inputTypeTag0 != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
+ throw new TypeMismatchException(AsterixBuiltinFunctions.REMOVE_FIELDS, 0, inputTypeTag0,
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG);
}
- if (inputArg1.getByteArray()[inputArg1.getStartOffset()] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
- throw new AlgebricksException(
- AsterixBuiltinFunctions.REMOVE_FIELDS.getName() + ": expects input type " + inputListType
- + ", but got " + EnumDeserializer.ATYPETAGDESERIALIZER
- .deserialize(inputArg1.getByteArray()[inputArg1.getStartOffset()]));
+ byte inputTypeTag1 = inputArg1.getByteArray()[inputArg1.getStartOffset()];
+ if (inputTypeTag1 != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
+ throw new TypeMismatchException(AsterixBuiltinFunctions.REMOVE_FIELDS, 1, inputTypeTag1,
+ ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
}
vp0.set(inputArg0);
@@ -122,14 +119,14 @@
processRecord(requiredRecType, recordPointable, listPointable, 0);
rbStack.get(0).write(out, true);
} catch (IOException | AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
}
private void processRecord(ARecordType requiredType, ARecordVisitablePointable srp,
AListVisitablePointable inputList, int nestedLevel)
- throws IOException, AsterixException, AlgebricksException {
+ throws IOException, AsterixException, HyracksDataException {
if (rbStack.size() < (nestedLevel + 1)) {
rbStack.add(new RecordBuilder());
}
@@ -160,7 +157,7 @@
private void addKeptFieldToSubRecord(ARecordType requiredType, IVisitablePointable fieldNamePointable,
IVisitablePointable fieldValuePointable, IVisitablePointable fieldTypePointable,
AListVisitablePointable inputList, int nestedLevel)
- throws IOException, AsterixException, AlgebricksException {
+ throws IOException, AsterixException, HyracksDataException {
runtimeRecordTypeInfo.reset(requiredType);
int pos = runtimeRecordTypeInfo.getFieldIndex(fieldNamePointable.getByteArray(),
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractIntervalLogicFuncDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractIntervalLogicFuncDescriptor.java
index b016fac..f782602 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractIntervalLogicFuncDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractIntervalLogicFuncDescriptor.java
@@ -22,12 +22,12 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -42,14 +42,12 @@
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
@@ -73,34 +71,31 @@
.getSerializerDeserializer(BuiltinType.ABOOLEAN);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
- try {
- if (argPtr0.getTag() != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG
- || argPtr1.getTag() != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG) {
- throw new AlgebricksException(getIdentifier().getName()
- + ": expects input type (INTERVAL, INTERVAL) but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argPtr0.getTag()) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argPtr1.getTag()) + ")");
- }
-
- argPtr0.getValue(interval0);
- argPtr1.getValue(interval1);
-
- if (interval0.getType() != interval1.getType()) {
- throw new AlgebricksException(getIdentifier().getName()
- + ": failed to compare intervals with different internal time type.");
- }
-
- ABoolean res = compareIntervals(il, interval0, interval1) ? ABoolean.TRUE : ABoolean.FALSE;
-
- booleanSerde.serialize(res, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ byte typeTag0 = argPtr0.getTag();
+ if (typeTag0 != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, typeTag0,
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
+ byte typeTag1 = argPtr0.getTag();
+ if (typeTag1 != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, typeTag1,
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
+ }
+
+ argPtr0.getValue(interval0);
+ argPtr1.getValue(interval1);
+
+ if (typeTag0 != typeTag1) {
+ throw new IncompatibleTypeException(getIdentifier(), typeTag0, typeTag1);
+ }
+
+ ABoolean res = compareIntervals(il, interval0, interval1) ? ABoolean.TRUE : ABoolean.FALSE;
+ booleanSerde.serialize(res, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java
index 1ce0c9c..a01d7a8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java
@@ -19,22 +19,24 @@
package org.apache.asterix.runtime.evaluators.functions.temporal;
import java.io.DataOutput;
+import java.io.IOException;
import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
import org.apache.asterix.om.base.temporal.ATimeParserFactory;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem.Fields;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -58,14 +60,12 @@
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
*/
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -82,7 +82,7 @@
private final UTF8StringWriter utf8Writer = new UTF8StringWriter();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -95,15 +95,13 @@
try {
if (bytes0[offset0] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects type DATETIME/NULL for parameter 0 but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]));
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects type STRING/NULL for parameter 1 but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]));
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
utf8Ptr.set(bytes1, offset1 + 1, len1 - 1);
@@ -111,7 +109,7 @@
utf8Ptr.getCharStartOffset());
if (!calInstance.validateTimeZone(timezone)) {
- throw new AlgebricksException(FID.getName() + ": wrong format for a time zone string!");
+ throw new InvalidDataFormatException(getIdentifier(), "timezone");
}
long chronon = ADateTimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
@@ -124,8 +122,8 @@
out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
utf8Writer.writeUTF8(sbder.toString(), out);
- } catch (Exception e1) {
- throw new AlgebricksException(e1);
+ } catch (IOException e1) {
+ throw new HyracksDataException(e1);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java
index bab00bd..9a2e91f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java
@@ -19,6 +19,7 @@
package org.apache.asterix.runtime.evaluators.functions.temporal;
import java.io.DataOutput;
+import java.io.IOException;
import org.apache.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
import org.apache.asterix.om.base.temporal.ATimeParserFactory;
@@ -28,13 +29,14 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -57,14 +59,13 @@
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
*/
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -79,7 +80,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -92,22 +93,20 @@
try {
if (bytes0[offset0] != ATypeTag.SERIALIZED_TIME_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects type TIME/NULL for parameter 0 but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]));
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_TIME_TYPE_TAG);
}
if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects type STRING/NULL for parameter 1 but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]));
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
utf8Ptr.set(argPtr1.getByteArray(), offset1 + 1, len1 - 1);
int timezone = ATimeParserFactory.parseTimezonePart(bytes1, utf8Ptr.getCharStartOffset());
if (!calInstance.validateTimeZone(timezone)) {
- throw new AlgebricksException(FID.getName() + ": wrong format for a time zone string!");
+ throw new InvalidDataFormatException(getIdentifier(), "timezone");
}
int chronon = ATimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
@@ -121,9 +120,8 @@
out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
writer.writeUTF8(sbder.toString(), out);
-
- } catch (Exception e1) {
- throw new AlgebricksException(e1);
+ } catch (IOException e1) {
+ throw new HyracksDataException(e1);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java
index e38da7e..985fd53 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java
@@ -32,9 +32,8 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -57,18 +56,14 @@
}
};
- /* (non-Javadoc)
- * @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
- */
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
@@ -80,15 +75,15 @@
private IScalarEvaluator eval1 = args[1].createScalarEvaluator(ctx);
@SuppressWarnings("unchecked")
- private ISerializerDeserializer<ADuration> durationSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADURATION);
+ private ISerializerDeserializer<ADuration> durationSerde = AqlSerializerDeserializerProvider.
+ INSTANCE.getSerializerDeserializer(BuiltinType.ADURATION);
private AMutableDuration aDuration = new AMutableDuration(0, 0);
private GregorianCalendarSystem calInstanct = GregorianCalendarSystem.getInstance();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -98,119 +93,114 @@
byte[] bytes1 = argPtr1.getByteArray();
int offset1 = argPtr1.getStartOffset();
- try {
- if (bytes0[offset0] != ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects type DATE/NULL for parameter 0 but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]));
- }
-
- if (bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects type DURATION/NULL for parameter 1 but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]));
- }
-
- int yearMonthDurationInMonths = ADurationSerializerDeserializer.getYearMonth(bytes1,
- offset1 + 1);
- long dayTimeDurationInMs = ADurationSerializerDeserializer.getDayTime(bytes1, offset1 + 1);
-
- long startingTimePoint = ADateSerializerDeserializer.getChronon(bytes0, offset0 + 1)
- * GregorianCalendarSystem.CHRONON_OF_DAY;
-
- long endingTimePoint = DurationArithmeticOperations.addDuration(startingTimePoint,
- yearMonthDurationInMonths, dayTimeDurationInMs, false);
-
- if (startingTimePoint == endingTimePoint) {
- aDuration.setValue(0, 0);
- } else {
-
- boolean negative = false;
-
- if (endingTimePoint < startingTimePoint) {
- negative = true;
- // swap the starting and ending time, so that ending time is always larger than the starting time.
- long tmpTime = endingTimePoint;
- endingTimePoint = startingTimePoint;
- startingTimePoint = tmpTime;
- }
-
- int year0 = calInstanct.getYear(startingTimePoint);
- int month0 = calInstanct.getMonthOfYear(startingTimePoint, year0);
-
- int year1 = calInstanct.getYear(endingTimePoint);
- int month1 = calInstanct.getMonthOfYear(endingTimePoint, year1);
-
- int year = year1 - year0;
- int month = month1 - month0;
- int day = calInstanct.getDayOfMonthYear(endingTimePoint, year1, month1)
- - calInstanct.getDayOfMonthYear(startingTimePoint, year0, month0);
- int hour = calInstanct.getHourOfDay(endingTimePoint)
- - calInstanct.getHourOfDay(startingTimePoint);
- int min = calInstanct.getMinOfHour(endingTimePoint)
- - calInstanct.getMinOfHour(startingTimePoint);
- int sec = calInstanct.getSecOfMin(endingTimePoint)
- - calInstanct.getSecOfMin(startingTimePoint);
- int ms = calInstanct.getMillisOfSec(endingTimePoint)
- - calInstanct.getMillisOfSec(startingTimePoint);
-
- if (ms < 0) {
- ms += GregorianCalendarSystem.CHRONON_OF_SECOND;
- sec -= 1;
- }
-
- if (sec < 0) {
- sec += GregorianCalendarSystem.CHRONON_OF_MINUTE
- / GregorianCalendarSystem.CHRONON_OF_SECOND;
- min -= 1;
- }
-
- if (min < 0) {
- min += GregorianCalendarSystem.CHRONON_OF_HOUR
- / GregorianCalendarSystem.CHRONON_OF_MINUTE;
- hour -= 1;
- }
-
- if (hour < 0) {
- hour += GregorianCalendarSystem.CHRONON_OF_DAY
- / GregorianCalendarSystem.CHRONON_OF_HOUR;
- day -= 1;
- }
-
- if (day < 0) {
- boolean isLeapYear = calInstanct.isLeapYear(year1);
- // need to "borrow" the days in previous month to make the day positive; when month is 1 (Jan), Dec will be borrowed
- day += (isLeapYear)
- ? (GregorianCalendarSystem.DAYS_OF_MONTH_LEAP[(12 + month1 - 2) % 12])
- : (GregorianCalendarSystem.DAYS_OF_MONTH_ORDI[(12 + month1 - 2) % 12]);
- month -= 1;
- }
-
- if (month < 0) {
- month += GregorianCalendarSystem.MONTHS_IN_A_YEAR;
- year -= 1;
- }
-
- if (negative) {
- aDuration.setValue(-1 * (year * GregorianCalendarSystem.MONTHS_IN_A_YEAR + month),
- -1 * (day * GregorianCalendarSystem.CHRONON_OF_DAY
- + hour * GregorianCalendarSystem.CHRONON_OF_HOUR
- + min * GregorianCalendarSystem.CHRONON_OF_MINUTE
- + sec * GregorianCalendarSystem.CHRONON_OF_SECOND + ms));
- } else {
- aDuration.setValue(year * GregorianCalendarSystem.MONTHS_IN_A_YEAR + month,
- day * GregorianCalendarSystem.CHRONON_OF_DAY
- + hour * GregorianCalendarSystem.CHRONON_OF_HOUR
- + min * GregorianCalendarSystem.CHRONON_OF_MINUTE
- + sec * GregorianCalendarSystem.CHRONON_OF_SECOND + ms);
- }
- }
-
- durationSerde.serialize(aDuration, out);
-
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ if (bytes0[offset0] != ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG);
}
+
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
+ }
+
+ int yearMonthDurationInMonths = ADurationSerializerDeserializer.getYearMonth(bytes1,
+ offset1 + 1);
+ long dayTimeDurationInMs = ADurationSerializerDeserializer.getDayTime(bytes1, offset1 + 1);
+
+ long startingTimePoint = ADateSerializerDeserializer.getChronon(bytes0, offset0 + 1)
+ * GregorianCalendarSystem.CHRONON_OF_DAY;
+
+ long endingTimePoint = DurationArithmeticOperations.addDuration(startingTimePoint,
+ yearMonthDurationInMonths, dayTimeDurationInMs, false);
+
+ if (startingTimePoint == endingTimePoint) {
+ aDuration.setValue(0, 0);
+ } else {
+
+ boolean negative = false;
+
+ if (endingTimePoint < startingTimePoint) {
+ negative = true;
+ // swap the starting and ending time, so that ending time is always larger than the
+ // starting time.
+ long tmpTime = endingTimePoint;
+ endingTimePoint = startingTimePoint;
+ startingTimePoint = tmpTime;
+ }
+
+ int year0 = calInstanct.getYear(startingTimePoint);
+ int month0 = calInstanct.getMonthOfYear(startingTimePoint, year0);
+
+ int year1 = calInstanct.getYear(endingTimePoint);
+ int month1 = calInstanct.getMonthOfYear(endingTimePoint, year1);
+
+ int year = year1 - year0;
+ int month = month1 - month0;
+ int day = calInstanct.getDayOfMonthYear(endingTimePoint, year1, month1)
+ - calInstanct.getDayOfMonthYear(startingTimePoint, year0, month0);
+ int hour = calInstanct.getHourOfDay(endingTimePoint)
+ - calInstanct.getHourOfDay(startingTimePoint);
+ int min = calInstanct.getMinOfHour(endingTimePoint)
+ - calInstanct.getMinOfHour(startingTimePoint);
+ int sec = calInstanct.getSecOfMin(endingTimePoint)
+ - calInstanct.getSecOfMin(startingTimePoint);
+ int ms = calInstanct.getMillisOfSec(endingTimePoint)
+ - calInstanct.getMillisOfSec(startingTimePoint);
+
+ if (ms < 0) {
+ ms += GregorianCalendarSystem.CHRONON_OF_SECOND;
+ sec -= 1;
+ }
+
+ if (sec < 0) {
+ sec += GregorianCalendarSystem.CHRONON_OF_MINUTE
+ / GregorianCalendarSystem.CHRONON_OF_SECOND;
+ min -= 1;
+ }
+
+ if (min < 0) {
+ min += GregorianCalendarSystem.CHRONON_OF_HOUR
+ / GregorianCalendarSystem.CHRONON_OF_MINUTE;
+ hour -= 1;
+ }
+
+ if (hour < 0) {
+ hour += GregorianCalendarSystem.CHRONON_OF_DAY
+ / GregorianCalendarSystem.CHRONON_OF_HOUR;
+ day -= 1;
+ }
+
+ if (day < 0) {
+ boolean isLeapYear = calInstanct.isLeapYear(year1);
+ // need to "borrow" the days in previous month to make the day positive; when month is
+ // 1 (Jan), Dec will be borrowed
+ day += isLeapYear
+ ? (GregorianCalendarSystem.DAYS_OF_MONTH_LEAP[(12 + month1 - 2) % 12])
+ : (GregorianCalendarSystem.DAYS_OF_MONTH_ORDI[(12 + month1 - 2) % 12]);
+ month -= 1;
+ }
+
+ if (month < 0) {
+ month += GregorianCalendarSystem.MONTHS_IN_A_YEAR;
+ year -= 1;
+ }
+
+ if (negative) {
+ aDuration.setValue(-1 * (year * GregorianCalendarSystem.MONTHS_IN_A_YEAR + month),
+ -1 * (day * GregorianCalendarSystem.CHRONON_OF_DAY
+ + hour * GregorianCalendarSystem.CHRONON_OF_HOUR
+ + min * GregorianCalendarSystem.CHRONON_OF_MINUTE
+ + sec * GregorianCalendarSystem.CHRONON_OF_SECOND + ms));
+ } else {
+ aDuration.setValue(year * GregorianCalendarSystem.MONTHS_IN_A_YEAR + month,
+ day * GregorianCalendarSystem.CHRONON_OF_DAY
+ + hour * GregorianCalendarSystem.CHRONON_OF_HOUR
+ + min * GregorianCalendarSystem.CHRONON_OF_MINUTE
+ + sec * GregorianCalendarSystem.CHRONON_OF_SECOND + ms);
+ }
+ }
+
+ durationSerde.serialize(aDuration, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java
index b053d99..cba7f20 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java
@@ -32,9 +32,8 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -55,9 +54,11 @@
* 2. Calculate the differences by fields between two different time points;<br/>
* 3. Re-format the duration into a human-readable one.
* <p/>
- * Here "human-readable" means the value of each field of the duration is within the value range of the field in the calendar system. For example, month would be in [0, 12), and hour would be in [0, 24).
+ * Here "human-readable" means the value of each field of the duration is within the value range of the field in the
+ * calendar system. For example, month would be in [0, 12), and hour would be in [0, 24).
* <p/>
- * The result can be considered as a "field-based" difference between the two datetime value, but all negative values would be converted to be non-negative.
+ * The result can be considered as a "field-based" difference between the two datetime value, but all negative values
+ * would be converted to be non-negative.
* <p/>
* In the implementation, we always do the subtraction from the later time point, resulting a positive duration always.
* <p/>
@@ -73,18 +74,14 @@
}
};
- /* (non-Javadoc)
- * @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
- */
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -95,15 +92,15 @@
private IScalarEvaluator eval1 = args[1].createScalarEvaluator(ctx);
@SuppressWarnings("unchecked")
- private ISerializerDeserializer<ADuration> durationSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADURATION);
+ private ISerializerDeserializer<ADuration> durationSerde = AqlSerializerDeserializerProvider.
+ INSTANCE.getSerializerDeserializer(BuiltinType.ADURATION);
private AMutableDuration aDuration = new AMutableDuration(0, 0);
private GregorianCalendarSystem calInstanct = GregorianCalendarSystem.getInstance();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -113,118 +110,110 @@
byte[] bytes1 = argPtr1.getByteArray();
int offset1 = argPtr1.getStartOffset();
- try {
- if (bytes0[offset0] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects type DATETIME/NULL for parameter 0 but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]));
- }
-
- if (bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects type DURATION/NULL for parameter 1 but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]));
- }
-
- int yearMonthDurationInMonths = ADurationSerializerDeserializer.getYearMonth(bytes1,
- offset1 + 1);
- long dayTimeDurationInMs = ADurationSerializerDeserializer.getDayTime(bytes1, offset1 + 1);
-
- long startingTimePoint = ADateTimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
-
- long endingTimePoint = DurationArithmeticOperations.addDuration(startingTimePoint,
- yearMonthDurationInMonths, dayTimeDurationInMs, false);
-
- if (startingTimePoint == endingTimePoint) {
- aDuration.setValue(0, 0);
- } else {
-
- boolean negative = false;
-
- if (endingTimePoint < startingTimePoint) {
- negative = true;
- // swap the starting and ending time, so that ending time is always larger than the starting time.
- long tmpTime = endingTimePoint;
- endingTimePoint = startingTimePoint;
- startingTimePoint = tmpTime;
- }
-
- int year0 = calInstanct.getYear(startingTimePoint);
- int month0 = calInstanct.getMonthOfYear(startingTimePoint, year0);
-
- int year1 = calInstanct.getYear(endingTimePoint);
- int month1 = calInstanct.getMonthOfYear(endingTimePoint, year1);
-
- int year = year1 - year0;
- int month = month1 - month0;
- int day = calInstanct.getDayOfMonthYear(endingTimePoint, year1, month1)
- - calInstanct.getDayOfMonthYear(startingTimePoint, year0, month0);
- int hour = calInstanct.getHourOfDay(endingTimePoint)
- - calInstanct.getHourOfDay(startingTimePoint);
- int min = calInstanct.getMinOfHour(endingTimePoint)
- - calInstanct.getMinOfHour(startingTimePoint);
- int sec = calInstanct.getSecOfMin(endingTimePoint)
- - calInstanct.getSecOfMin(startingTimePoint);
- int ms = calInstanct.getMillisOfSec(endingTimePoint)
- - calInstanct.getMillisOfSec(startingTimePoint);
-
- if (ms < 0) {
- ms += GregorianCalendarSystem.CHRONON_OF_SECOND;
- sec -= 1;
- }
-
- if (sec < 0) {
- sec += GregorianCalendarSystem.CHRONON_OF_MINUTE
- / GregorianCalendarSystem.CHRONON_OF_SECOND;
- min -= 1;
- }
-
- if (min < 0) {
- min += GregorianCalendarSystem.CHRONON_OF_HOUR
- / GregorianCalendarSystem.CHRONON_OF_MINUTE;
- hour -= 1;
- }
-
- if (hour < 0) {
- hour += GregorianCalendarSystem.CHRONON_OF_DAY
- / GregorianCalendarSystem.CHRONON_OF_HOUR;
- day -= 1;
- }
-
- if (day < 0) {
- boolean isLeapYear = calInstanct.isLeapYear(year1);
- // need to "borrow" the days in previous month to make the day positive; when month is 1 (Jan), Dec will be borrowed
- day += (isLeapYear)
- ? (GregorianCalendarSystem.DAYS_OF_MONTH_LEAP[(12 + month1 - 2) % 12])
- : (GregorianCalendarSystem.DAYS_OF_MONTH_ORDI[(12 + month1 - 2) % 12]);
- month -= 1;
- }
-
- if (month < 0) {
- month += GregorianCalendarSystem.MONTHS_IN_A_YEAR;
- year -= 1;
- }
-
- if (negative) {
- aDuration.setValue(-1 * (year * GregorianCalendarSystem.MONTHS_IN_A_YEAR + month),
- -1 * (day * GregorianCalendarSystem.CHRONON_OF_DAY
- + hour * GregorianCalendarSystem.CHRONON_OF_HOUR
- + min * GregorianCalendarSystem.CHRONON_OF_MINUTE
- + sec * GregorianCalendarSystem.CHRONON_OF_SECOND + ms));
- } else {
- aDuration.setValue(year * GregorianCalendarSystem.MONTHS_IN_A_YEAR + month,
- day * GregorianCalendarSystem.CHRONON_OF_DAY
- + hour * GregorianCalendarSystem.CHRONON_OF_HOUR
- + min * GregorianCalendarSystem.CHRONON_OF_MINUTE
- + sec * GregorianCalendarSystem.CHRONON_OF_SECOND + ms);
- }
- }
-
- durationSerde.serialize(aDuration, out);
-
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ if (bytes0[offset0] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
+
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
+ }
+
+ int yearMonthDurationInMonths = ADurationSerializerDeserializer.getYearMonth(bytes1,
+ offset1 + 1);
+ long dayTimeDurationInMs = ADurationSerializerDeserializer.getDayTime(bytes1, offset1 + 1);
+
+ long startingTimePoint = ADateTimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
+
+ long endingTimePoint = DurationArithmeticOperations.addDuration(startingTimePoint,
+ yearMonthDurationInMonths, dayTimeDurationInMs, false);
+
+ if (startingTimePoint == endingTimePoint) {
+ aDuration.setValue(0, 0);
+ } else {
+ boolean negative = false;
+ if (endingTimePoint < startingTimePoint) {
+ negative = true;
+ // swap the starting and ending time, so that ending time is always larger than the
+ // starting time.
+ long tmpTime = endingTimePoint;
+ endingTimePoint = startingTimePoint;
+ startingTimePoint = tmpTime;
+ }
+
+ int year0 = calInstanct.getYear(startingTimePoint);
+ int month0 = calInstanct.getMonthOfYear(startingTimePoint, year0);
+
+ int year1 = calInstanct.getYear(endingTimePoint);
+ int month1 = calInstanct.getMonthOfYear(endingTimePoint, year1);
+
+ int year = year1 - year0;
+ int month = month1 - month0;
+ int day = calInstanct.getDayOfMonthYear(endingTimePoint, year1, month1)
+ - calInstanct.getDayOfMonthYear(startingTimePoint, year0, month0);
+ int hour = calInstanct.getHourOfDay(endingTimePoint)
+ - calInstanct.getHourOfDay(startingTimePoint);
+ int min = calInstanct.getMinOfHour(endingTimePoint)
+ - calInstanct.getMinOfHour(startingTimePoint);
+ int sec = calInstanct.getSecOfMin(endingTimePoint)
+ - calInstanct.getSecOfMin(startingTimePoint);
+ int ms = calInstanct.getMillisOfSec(endingTimePoint)
+ - calInstanct.getMillisOfSec(startingTimePoint);
+
+ if (ms < 0) {
+ ms += GregorianCalendarSystem.CHRONON_OF_SECOND;
+ sec -= 1;
+ }
+
+ if (sec < 0) {
+ sec += GregorianCalendarSystem.CHRONON_OF_MINUTE
+ / GregorianCalendarSystem.CHRONON_OF_SECOND;
+ min -= 1;
+ }
+
+ if (min < 0) {
+ min += GregorianCalendarSystem.CHRONON_OF_HOUR
+ / GregorianCalendarSystem.CHRONON_OF_MINUTE;
+ hour -= 1;
+ }
+
+ if (hour < 0) {
+ hour += GregorianCalendarSystem.CHRONON_OF_DAY
+ / GregorianCalendarSystem.CHRONON_OF_HOUR;
+ day -= 1;
+ }
+
+ if (day < 0) {
+ boolean isLeapYear = calInstanct.isLeapYear(year1);
+ // need to "borrow" the days in previous month to make the day positive; when month is
+ // 1 (Jan), Dec will be borrowed
+ day += isLeapYear
+ ? (GregorianCalendarSystem.DAYS_OF_MONTH_LEAP[(12 + month1 - 2) % 12])
+ : (GregorianCalendarSystem.DAYS_OF_MONTH_ORDI[(12 + month1 - 2) % 12]);
+ month -= 1;
+ }
+
+ if (month < 0) {
+ month += GregorianCalendarSystem.MONTHS_IN_A_YEAR;
+ year -= 1;
+ }
+
+ if (negative) {
+ aDuration.setValue(-1 * (year * GregorianCalendarSystem.MONTHS_IN_A_YEAR + month),
+ -1 * (day * GregorianCalendarSystem.CHRONON_OF_DAY
+ + hour * GregorianCalendarSystem.CHRONON_OF_HOUR
+ + min * GregorianCalendarSystem.CHRONON_OF_MINUTE
+ + sec * GregorianCalendarSystem.CHRONON_OF_SECOND + ms));
+ } else {
+ aDuration.setValue(year * GregorianCalendarSystem.MONTHS_IN_A_YEAR + month,
+ day * GregorianCalendarSystem.CHRONON_OF_DAY
+ + hour * GregorianCalendarSystem.CHRONON_OF_HOUR
+ + min * GregorianCalendarSystem.CHRONON_OF_MINUTE
+ + sec * GregorianCalendarSystem.CHRONON_OF_SECOND + ms);
+ }
+ }
+ durationSerde.serialize(aDuration, out);
result.set(resultStorage);
}
};
@@ -233,8 +222,8 @@
}
/* (non-Javadoc)
- * @see org.apache.asterix.om.functions.IFunctionDescriptor#getIdentifier()
- */
+ * @see org.apache.asterix.om.functions.IFunctionDescriptor#getIdentifier()
+ */
@Override
public FunctionIdentifier getIdentifier() {
return FID;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateDescriptor.java
index e1f783a..e958a30 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateDescriptor.java
@@ -29,7 +29,6 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -54,13 +53,13 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -72,17 +71,12 @@
private AMutableDate aDate = new AMutableDate(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
- try {
- resultStorage.reset();
- int dateChronon = (int) (System.currentTimeMillis()
- / GregorianCalendarSystem.CHRONON_OF_DAY);
- aDate.setValue(dateChronon);
- dateSerde.serialize(aDate, out);
- result.set(resultStorage);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
- }
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ int dateChronon = (int) (System.currentTimeMillis() / GregorianCalendarSystem.CHRONON_OF_DAY);
+ aDate.setValue(dateChronon);
+ dateSerde.serialize(aDate, out);
+ result.set(resultStorage);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateTimeDescriptor.java
index c7cc45d..c8f310a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateTimeDescriptor.java
@@ -28,7 +28,6 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -56,13 +55,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -74,15 +72,11 @@
private AMutableDateTime aDateTime = new AMutableDateTime(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
- try {
- resultStorage.reset();
- aDateTime.setValue(System.currentTimeMillis());
- datetimeSerde.serialize(aDateTime, out);
- result.set(resultStorage);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
- }
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ aDateTime.setValue(System.currentTimeMillis());
+ datetimeSerde.serialize(aDateTime, out);
+ result.set(resultStorage);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentTimeDescriptor.java
index 857c25b..be1f877 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentTimeDescriptor.java
@@ -29,7 +29,6 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -57,13 +56,12 @@
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
*/
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -75,17 +73,12 @@
private AMutableTime aTime = new AMutableTime(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
- try {
- resultStorage.reset();
- int timeChronon = (int) (System.currentTimeMillis()
- % GregorianCalendarSystem.CHRONON_OF_DAY);
- aTime.setValue(timeChronon);
- timeSerde.serialize(aTime, out);
- result.set(resultStorage);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
- }
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ int timeChronon = (int) (System.currentTimeMillis() % GregorianCalendarSystem.CHRONON_OF_DAY);
+ aTime.setValue(timeChronon);
+ timeSerde.serialize(aTime, out);
+ result.set(resultStorage);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromDatetimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromDatetimeDescriptor.java
index d4fa0e6..869ac7b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromDatetimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromDatetimeDescriptor.java
@@ -25,14 +25,13 @@
import org.apache.asterix.om.base.ADate;
import org.apache.asterix.om.base.AMutableDate;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -57,14 +56,13 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -79,29 +77,23 @@
private AMutableDate aDate = new AMutableDate(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int offset = argPtr.getStartOffset();
- try {
- if (bytes[offset] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects input type DATETIME/NULL but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]));
- }
- long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
- int dateChrononInDays = (int) (datetimeChronon / GregorianCalendarSystem.CHRONON_OF_DAY);
- if (dateChrononInDays < 0
- && datetimeChronon % GregorianCalendarSystem.CHRONON_OF_DAY != 0) {
- dateChrononInDays -= 1;
- }
- aDate.setValue(dateChrononInDays);
- dateSerde.serialize(aDate, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ if (bytes[offset] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
+ long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
+ int dateChrononInDays = (int) (datetimeChronon / GregorianCalendarSystem.CHRONON_OF_DAY);
+ if (dateChrononInDays < 0 && datetimeChronon % GregorianCalendarSystem.CHRONON_OF_DAY != 0) {
+ dateChrononInDays -= 1;
+ }
+ aDate.setValue(dateChrononInDays);
+ dateSerde.serialize(aDate, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromUnixTimeInDaysDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromUnixTimeInDaysDescriptor.java
index 5fa57f6..58e2213 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromUnixTimeInDaysDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromUnixTimeInDaysDescriptor.java
@@ -29,7 +29,6 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -54,14 +53,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -77,16 +74,12 @@
.getSerializerDeserializer(BuiltinType.ADATE);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval.evaluate(tuple, argPtr);
- try {
- aDate.setValue(
- ATypeHierarchy.getIntegerValue(argPtr.getByteArray(), argPtr.getStartOffset()));
- dateSerde.serialize(aDate, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
- }
+ aDate.setValue(ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0,
+ argPtr.getByteArray(), argPtr.getStartOffset()));
+ dateSerde.serialize(aDate, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromDateAndTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromDateAndTimeDescriptor.java
index 98f84aa..96a3e30 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromDateAndTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromDateAndTimeDescriptor.java
@@ -26,14 +26,13 @@
import org.apache.asterix.om.base.ADateTime;
import org.apache.asterix.om.base.AMutableDateTime;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -61,14 +60,12 @@
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
*/
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -85,7 +82,7 @@
private AMutableDateTime aDateTime = new AMutableDateTime(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -95,25 +92,21 @@
byte[] bytes1 = argPtr1.getByteArray();
int offset1 = argPtr1.getStartOffset();
- try {
- if (bytes0[offset0] != ATypeTag.SERIALIZED_DATE_TYPE_TAG
- && bytes1[offset1] != ATypeTag.SERIALIZED_TIME_TYPE_TAG) {
- throw new AlgebricksException(FID.getName()
- + ": expects input type (DATE/NULL, TIME/NULL) but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]) + ").");
-
- }
-
- long datetimeChronon = ADateSerializerDeserializer.getChronon(bytes0, offset0 + 1)
- * GregorianCalendarSystem.CHRONON_OF_DAY
- + ATimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
-
- aDateTime.setValue(datetimeChronon);
- datetimeSerde.serialize(aDateTime, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ if (bytes0[offset0] != ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG);
}
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_TIME_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+ }
+
+ long datetimeChronon = ADateSerializerDeserializer.getChronon(bytes0, offset0 + 1)
+ * GregorianCalendarSystem.CHRONON_OF_DAY
+ + ATimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
+
+ aDateTime.setValue(datetimeChronon);
+ datetimeSerde.serialize(aDateTime, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsDescriptor.java
index d9c0156..a5200cd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsDescriptor.java
@@ -27,13 +27,13 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.ADateTime;
import org.apache.asterix.om.base.AMutableDateTime;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -62,14 +62,13 @@
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
*/
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -85,34 +84,31 @@
private AMutableDateTime aDatetime = new AMutableDateTime(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int offset = argPtr.getStartOffset();
- try {
- ATypeTag argPtrTypeTag = ATypeTag.VALUE_TYPE_MAPPING[bytes[offset]];
- switch (argPtrTypeTag) {
- case INT8:
- aDatetime.setValue(AInt8SerializerDeserializer.getByte(bytes, offset + 1));
- break;
- case INT16:
- aDatetime.setValue(AInt16SerializerDeserializer.getShort(bytes, offset + 1));
- break;
- case INT32:
- aDatetime.setValue(AInt32SerializerDeserializer.getInt(bytes, offset + 1));
- break;
- case INT64:
- aDatetime.setValue(AInt64SerializerDeserializer.getLong(bytes, offset + 1));
- break;
- default:
- throw new AlgebricksException(FID.getName()
- + ": expects type INT8/INT16/INT32/INT64/NULL but got " + argPtrTypeTag);
- }
- datetimeSerde.serialize(aDatetime, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ ATypeTag argPtrTypeTag = ATypeTag.VALUE_TYPE_MAPPING[bytes[offset]];
+ switch (argPtrTypeTag) {
+ case INT8:
+ aDatetime.setValue(AInt8SerializerDeserializer.getByte(bytes, offset + 1));
+ break;
+ case INT16:
+ aDatetime.setValue(AInt16SerializerDeserializer.getShort(bytes, offset + 1));
+ break;
+ case INT32:
+ aDatetime.setValue(AInt32SerializerDeserializer.getInt(bytes, offset + 1));
+ break;
+ case INT64:
+ aDatetime.setValue(AInt64SerializerDeserializer.getLong(bytes, offset + 1));
+ break;
+ default:
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+ ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
}
+ datetimeSerde.serialize(aDatetime, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsDescriptor.java
index a0258b6..b71ac93e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsDescriptor.java
@@ -7,7 +7,7 @@
* "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
+ *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
@@ -27,13 +27,13 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.ADateTime;
import org.apache.asterix.om.base.AMutableDateTime;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -62,14 +62,12 @@
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
*/
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -85,39 +83,33 @@
private AMutableDateTime aDatetime = new AMutableDateTime(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int offset = argPtr.getStartOffset();
+ ATypeTag argPtrTypeTag = ATypeTag.VALUE_TYPE_MAPPING[bytes[offset]];
- try {
-
- ATypeTag argPtrTypeTag = ATypeTag.VALUE_TYPE_MAPPING[bytes[offset]];
-
- switch (argPtrTypeTag) {
- case INT8:
- aDatetime.setValue(AInt8SerializerDeserializer.getByte(bytes, offset + 1) * 1000l);
- break;
- case INT16:
- aDatetime
- .setValue(AInt16SerializerDeserializer.getShort(bytes, offset + 1) * 1000l);
- break;
- case INT32:
- aDatetime.setValue(AInt32SerializerDeserializer.getInt(bytes, offset + 1) * 1000l);
- break;
- case INT64:
- aDatetime.setValue(AInt64SerializerDeserializer.getLong(bytes, offset + 1) * 1000l);
- break;
- default:
- throw new AlgebricksException(FID.getName()
- + ": expects type INT8/INT16/INT32/INT64/NULL but got " + argPtrTypeTag);
- }
- datetimeSerde.serialize(aDatetime, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ switch (argPtrTypeTag) {
+ case INT8:
+ aDatetime.setValue(AInt8SerializerDeserializer.getByte(bytes, offset + 1) * 1000l);
+ break;
+ case INT16:
+ aDatetime.setValue(AInt16SerializerDeserializer.getShort(bytes, offset + 1) * 1000l);
+ break;
+ case INT32:
+ aDatetime.setValue(AInt32SerializerDeserializer.getInt(bytes, offset + 1) * 1000l);
+ break;
+ case INT64:
+ aDatetime.setValue(AInt64SerializerDeserializer.getLong(bytes, offset + 1) * 1000l);
+ break;
+ default:
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+ ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
}
+ datetimeSerde.serialize(aDatetime, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java
index e1fe88b..5993da9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java
@@ -26,14 +26,13 @@
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -61,14 +60,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -83,53 +80,48 @@
private AMutableInt64 aInt64 = new AMutableInt64(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int offset = argPtr.getStartOffset();
- try {
- int daysSinceAnchor;
- int reminder = 0;
- if (bytes[offset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
- daysSinceAnchor = (int) (ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1)
- / GregorianCalendarSystem.CHRONON_OF_DAY);
- reminder = (int) (ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1)
- % GregorianCalendarSystem.CHRONON_OF_DAY);
- } else if (bytes[offset] == ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
- daysSinceAnchor = ADateSerializerDeserializer.getChronon(bytes, offset + 1);
- } else {
- throw new AlgebricksException(
- FID.getName() + ": expects input type DATETIME/DATE/NULL but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]));
- }
-
- // adjust the day before 1970-01-01
- if (daysSinceAnchor < 0 && reminder != 0) {
- daysSinceAnchor -= 1;
- }
-
- // compute the weekday (0-based, and 0 = Sunday). Adjustment is needed as
- // the anchor day is Thursday.
- int weekday = (daysSinceAnchor + ANCHOR_WEEKDAY) % 7;
-
- // handle the negative weekday
- if (weekday < 0) {
- weekday += 7;
- }
-
- // convert from 0-based to 1-based (so 7 = Sunday)
- if (weekday == 0) {
- weekday = 7;
- }
-
- aInt64.setValue(weekday);
- int64Serde.serialize(aInt64, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ int daysSinceAnchor;
+ int reminder = 0;
+ if (bytes[offset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
+ daysSinceAnchor = (int) (ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1)
+ / GregorianCalendarSystem.CHRONON_OF_DAY);
+ reminder = (int) (ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1)
+ % GregorianCalendarSystem.CHRONON_OF_DAY);
+ } else if (bytes[offset] == ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
+ daysSinceAnchor = ADateSerializerDeserializer.getChronon(bytes, offset + 1);
+ } else {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG, ATypeTag.SERIALIZED_DATE_TYPE_TAG);
}
+
+ // adjust the day before 1970-01-01
+ if (daysSinceAnchor < 0 && reminder != 0) {
+ daysSinceAnchor -= 1;
+ }
+
+ // compute the weekday (0-based, and 0 = Sunday). Adjustment is needed as
+ // the anchor day is Thursday.
+ int weekday = (daysSinceAnchor + ANCHOR_WEEKDAY) % 7;
+
+ // handle the negative weekday
+ if (weekday < 0) {
+ weekday += 7;
+ }
+
+ // convert from 0-based to 1-based (so 7 = Sunday)
+ if (weekday == 0) {
+ weekday = 7;
+ }
+
+ aInt64.setValue(weekday);
+ int64Serde.serialize(aInt64, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationComparatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationComparatorDescriptor.java
index 2d29c81..3188b08 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationComparatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationComparatorDescriptor.java
@@ -26,9 +26,9 @@
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -52,14 +52,12 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -74,7 +72,7 @@
.getSerializerDeserializer(BuiltinType.ABOOLEAN);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -84,29 +82,26 @@
byte[] bytes1 = argPtr1.getByteArray();
int offset1 = argPtr1.getStartOffset();
- try {
- if (bytes0[offset0] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG
- || bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
- throw new AlgebricksException(getIdentifier().getName()
- + ": expects type NULL/DURATION, NULL/DURATION but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + " and "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]));
- }
+ if (bytes0[offset0] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
+ }
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
+ }
- if ((ADurationSerializerDeserializer.getYearMonth(bytes0, offset0 + 1) != 0)
- || (ADurationSerializerDeserializer.getYearMonth(bytes1, offset1 + 1) != 0)) {
- throw new AlgebricksException(
- getIdentifier().getName() + ": only year-month durations are allowed.");
- }
+ if ((ADurationSerializerDeserializer.getYearMonth(bytes0, offset0 + 1) != 0)
+ || (ADurationSerializerDeserializer.getYearMonth(bytes1, offset1 + 1) != 0)) {
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG);
+ }
- if (ADurationSerializerDeserializer.getDayTime(bytes0,
- offset0 + 1) > ADurationSerializerDeserializer.getDayTime(bytes1, offset1 + 1)) {
- boolSerde.serialize(isGreaterThan ? ABoolean.TRUE : ABoolean.FALSE, out);
- } else {
- boolSerde.serialize(isGreaterThan ? ABoolean.FALSE : ABoolean.TRUE, out);
- }
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ if (ADurationSerializerDeserializer.getDayTime(bytes0,
+ offset0 + 1) > ADurationSerializerDeserializer.getDayTime(bytes1, offset1 + 1)) {
+ boolSerde.serialize(isGreaterThan ? ABoolean.TRUE : ABoolean.FALSE, out);
+ } else {
+ boolSerde.serialize(isGreaterThan ? ABoolean.FALSE : ABoolean.TRUE, out);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationEqualDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationEqualDescriptor.java
index 0a88684..27b3304 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationEqualDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationEqualDescriptor.java
@@ -23,14 +23,13 @@
import org.apache.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -54,14 +53,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -76,7 +73,7 @@
.getSerializerDeserializer(BuiltinType.ABOOLEAN);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -86,26 +83,23 @@
byte[] bytes1 = argPtr1.getByteArray();
int offset1 = argPtr1.getStartOffset();
- try {
- if (bytes0[offset0] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG
- || bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
- throw new AlgebricksException(FID.getName()
- + ": expects type NULL/DURATION, NULL/DURATION but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + " and "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]));
- }
+ if (bytes0[offset0] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
+ }
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
+ }
- if ((ADurationSerializerDeserializer.getDayTime(bytes0,
- offset0 + 1) == ADurationSerializerDeserializer.getDayTime(bytes1, offset1 + 1))
- && (ADurationSerializerDeserializer.getYearMonth(bytes0,
- offset0 + 1) == ADurationSerializerDeserializer.getYearMonth(bytes1,
+ if ((ADurationSerializerDeserializer.getDayTime(bytes0,
+ offset0 + 1) == ADurationSerializerDeserializer.getDayTime(bytes1, offset1 + 1))
+ && (ADurationSerializerDeserializer.getYearMonth(bytes0,
+ offset0 + 1) == ADurationSerializerDeserializer.getYearMonth(bytes1,
offset1 + 1))) {
boolSerde.serialize(ABoolean.TRUE, out);
- } else {
+ } else {
boolSerde.serialize(ABoolean.FALSE, out);
- }
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromIntervalDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromIntervalDescriptor.java
index 0c4042f..d3862b7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromIntervalDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromIntervalDescriptor.java
@@ -25,14 +25,13 @@
import org.apache.asterix.om.base.ADayTimeDuration;
import org.apache.asterix.om.base.AMutableDayTimeDuration;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -57,14 +56,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -79,35 +76,28 @@
private AMutableDayTimeDuration aDayTimeDuration = new AMutableDayTimeDuration(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int offset = argPtr.getStartOffset();
- try {
- if (bytes[offset] != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects INTERVAL/NULL as the input but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]));
- }
- long chrononStart = AIntervalSerializerDeserializer.getIntervalStart(bytes, offset + 1);
- long chrononEnd = AIntervalSerializerDeserializer.getIntervalEnd(bytes, offset + 1);
- byte intervalTypeTag = AIntervalSerializerDeserializer.getIntervalTimeType(bytes,
- offset + 1);
-
- if (intervalTypeTag == ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
- chrononStart *= GregorianCalendarSystem.CHRONON_OF_DAY;
- chrononEnd *= GregorianCalendarSystem.CHRONON_OF_DAY;
- }
-
- aDayTimeDuration.setMilliseconds(chrononEnd - chrononStart);
- dayTimeDurationSerde.serialize(aDayTimeDuration, out);
-
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ if (bytes[offset] != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
+ long chrononStart = AIntervalSerializerDeserializer.getIntervalStart(bytes, offset + 1);
+ long chrononEnd = AIntervalSerializerDeserializer.getIntervalEnd(bytes, offset + 1);
+ byte intervalTypeTag = AIntervalSerializerDeserializer.getIntervalTimeType(bytes, offset + 1);
+
+ if (intervalTypeTag == ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
+ chrononStart *= GregorianCalendarSystem.CHRONON_OF_DAY;
+ chrononEnd *= GregorianCalendarSystem.CHRONON_OF_DAY;
+ }
+
+ aDayTimeDuration.setMilliseconds(chrononEnd - chrononStart);
+ dayTimeDurationSerde.serialize(aDayTimeDuration, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMillisecondsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMillisecondsDescriptor.java
index 480cf8f..fa1e681 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMillisecondsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMillisecondsDescriptor.java
@@ -27,13 +27,13 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.ADuration;
import org.apache.asterix.om.base.AMutableDuration;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -59,14 +59,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -81,35 +79,32 @@
AMutableDuration aDuration = new AMutableDuration(0, 0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
byte[] bytes = argPtr0.getByteArray();
int offset = argPtr0.getStartOffset();
- try {
- ATypeTag argPtrTypeTag = ATypeTag.VALUE_TYPE_MAPPING[bytes[offset]];
- switch (argPtrTypeTag) {
- case INT8:
- aDuration.setValue(0, AInt8SerializerDeserializer.getByte(bytes, offset + 1));
- break;
- case INT16:
- aDuration.setValue(0, AInt16SerializerDeserializer.getShort(bytes, offset + 1));
- break;
- case INT32:
- aDuration.setValue(0, AInt32SerializerDeserializer.getInt(bytes, offset + 1));
- break;
- case INT64:
- aDuration.setValue(0, AInt64SerializerDeserializer.getLong(bytes, offset + 1));
- break;
- default:
- throw new AlgebricksException(FID.getName()
- + ": expects type INT8/INT16/INT32/INT64/NULL but got " + argPtrTypeTag);
- }
- durationSerde.serialize(aDuration, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ ATypeTag argPtrTypeTag = ATypeTag.VALUE_TYPE_MAPPING[bytes[offset]];
+ switch (argPtrTypeTag) {
+ case INT8:
+ aDuration.setValue(0, AInt8SerializerDeserializer.getByte(bytes, offset + 1));
+ break;
+ case INT16:
+ aDuration.setValue(0, AInt16SerializerDeserializer.getShort(bytes, offset + 1));
+ break;
+ case INT32:
+ aDuration.setValue(0, AInt32SerializerDeserializer.getInt(bytes, offset + 1));
+ break;
+ case INT64:
+ aDuration.setValue(0, AInt64SerializerDeserializer.getLong(bytes, offset + 1));
+ break;
+ default:
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+ ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
}
+ durationSerde.serialize(aDuration, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMonthsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMonthsDescriptor.java
index 5db6294..907992b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMonthsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMonthsDescriptor.java
@@ -29,7 +29,6 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -54,14 +53,13 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -76,19 +74,14 @@
AMutableDuration aDuration = new AMutableDuration(0, 0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
-
byte[] bytes = argPtr0.getByteArray();
int offset = argPtr0.getStartOffset();
-
- try {
- aDuration.setValue(ATypeHierarchy.getIntegerValue(bytes, offset), 0);
- durationSerde.serialize(aDuration, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
- }
+ aDuration.setValue(ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes, offset),
+ 0);
+ durationSerde.serialize(aDuration, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java
index cb84fc8..3f9616b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java
@@ -24,14 +24,13 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.ADayTimeDuration;
import org.apache.asterix.om.base.AMutableDayTimeDuration;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -55,14 +54,13 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -77,25 +75,21 @@
AMutableDayTimeDuration aDayTimeDuration = new AMutableDayTimeDuration(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
byte[] bytes = argPtr0.getByteArray();
int offset = argPtr0.getStartOffset();
- try {
- if (bytes[offset] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
- throw new AlgebricksException(FID.getName() + ": expects NULL/DURATION, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]));
- }
-
- aDayTimeDuration
- .setMilliseconds(ADurationSerializerDeserializer.getDayTime(bytes, offset + 1));
- dayTimeDurationSerde.serialize(aDayTimeDuration, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ if (bytes[offset] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
}
+
+ aDayTimeDuration
+ .setMilliseconds(ADurationSerializerDeserializer.getDayTime(bytes, offset + 1));
+ dayTimeDurationSerde.serialize(aDayTimeDuration, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
index d4e3b5f..8258df8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
@@ -30,9 +30,9 @@
import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -54,14 +54,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
protected final IntervalLogic il = new IntervalLogic();
@@ -88,45 +86,41 @@
.getSerializerDeserializer(BuiltinType.AINTERVAL);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
byte type0 = argPtr0.getTag();
byte type1 = argPtr1.getTag();
- try {
- if (type0 == ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG && type0 == type1) {
- argPtr0.getValue(interval0);
- argPtr1.getValue(interval1);
- byte intervalType0 = interval0.getType();
- byte intervalType1 = interval1.getType();
- if (intervalType0 != intervalType1) {
- throw new AlgebricksException(getIdentifier().getName()
- + ": expecting two (nullable) interval values with the same internal time type but got interval of "
- + interval0.getTypeTag() + " and interval of " + interval1.getTypeTag());
- }
+ if (type0 == ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG && type0 == type1) {
+ argPtr0.getValue(interval0);
+ argPtr1.getValue(interval1);
+ byte intervalType0 = interval0.getType();
+ byte intervalType1 = interval1.getType();
- if (il.overlaps(interval0, interval1) || il.overlappedBy(interval0, interval1)
- || il.covers(interval0, interval1) || il.coveredBy(interval0, interval1)) {
- long start = Math.max(interval0.getStartValue(), interval1.getStartValue());
- long end = Math.min(interval0.getEndValue(), interval1.getEndValue());
- aInterval.setValue(start, end, intervalType0);
- intervalSerde.serialize(aInterval, out);
- } else {
- nullSerde.serialize(ANull.NULL, out);
- }
- } else {
- throw new AlgebricksException(getIdentifier().getName()
- + ": expecting two (nullable) interval values but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(type0) + " and "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(type1));
+ if (intervalType0 != intervalType1) {
+ throw new IncompatibleTypeException(getIdentifier(), intervalType0, intervalType1);
}
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+
+ if (il.overlaps(interval0, interval1) || il.overlappedBy(interval0, interval1)
+ || il.covers(interval0, interval1) || il.coveredBy(interval0, interval1)) {
+ long start = Math.max(interval0.getStartValue(), interval1.getStartValue());
+ long end = Math.min(interval0.getEndValue(), interval1.getEndValue());
+ aInterval.setValue(start, end, intervalType0);
+ intervalSerde.serialize(aInterval, out);
+ } else {
+ nullSerde.serialize(ANull.NULL, out);
+ }
+ result.set(resultStorage);
+ return;
+ } else if (type0 != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, type0,
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
+ } else {
+ throw new IncompatibleTypeException(getIdentifier(), type0, type1);
}
- result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java
index d2e4b6e..7224c13 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java
@@ -24,14 +24,13 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.AMutableYearMonthDuration;
import org.apache.asterix.om.base.AYearMonthDuration;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -55,14 +54,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -77,25 +74,17 @@
AMutableYearMonthDuration aYearMonthDuration = new AMutableYearMonthDuration(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
-
byte[] bytes0 = argPtr0.getByteArray();
int offset0 = argPtr0.getStartOffset();
-
- try {
- if (bytes0[offset0] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
- throw new AlgebricksException(FID.getName() + ": expects NULL/DURATION, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]));
- }
-
- aYearMonthDuration
- .setMonths(ADurationSerializerDeserializer.getYearMonth(bytes0, offset0 + 1));
- yearMonthDurationSerde.serialize(aYearMonthDuration, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ if (bytes0[offset0] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
}
+ aYearMonthDuration.setMonths(ADurationSerializerDeserializer.getYearMonth(bytes0, offset0 + 1));
+ yearMonthDurationSerde.serialize(aYearMonthDuration, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java
index b584442..c8b27d5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java
@@ -37,7 +37,11 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.OverflowException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.runtime.exceptions.UnderflowException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -61,14 +65,13 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -91,7 +94,7 @@
private final GregorianCalendarSystem GREG_CAL = GregorianCalendarSystem.getInstance();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -114,8 +117,9 @@
chrononToBin = ADateTimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
break;
default:
- throw new AlgebricksException(getIdentifier().getName()
- + ": the first argument should be DATE/TIME/DATETIME/NULL but got " + type0);
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_TIME_TYPE_TAG,
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
byte[] bytes1 = argPtr1.getByteArray();
@@ -123,8 +127,7 @@
ATypeTag type1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
if (type0 != type1) {
- throw new AlgebricksException(getIdentifier().getName() + ": expecting " + type0
- + " for the second argument but got " + type1);
+ throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0], bytes1[offset1]);
}
long chrononToStart = 0;
@@ -140,8 +143,9 @@
chrononToStart = ADateTimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
break;
default:
- throw new AlgebricksException(getIdentifier().getName() + ": expecting " + type0
- + " for the second argument but got " + type1);
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_TIME_TYPE_TAG,
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
byte[] bytes2 = argPtr2.getByteArray();
@@ -165,13 +169,11 @@
+ ((totalMonths < 0 && totalMonths % yearMonth != 0) ? -1 : 0);
if (binIndex > Integer.MAX_VALUE) {
- throw new AlgebricksException(
- getIdentifier().getName() + ": Overflowing time value to be binned!");
+ throw new OverflowException(getIdentifier());
}
if (binIndex < Integer.MIN_VALUE) {
- throw new AlgebricksException(
- getIdentifier().getName() + ": Underflowing time value to be binned!");
+ throw new UnderflowException(getIdentifier());
}
break;
@@ -184,9 +186,9 @@
+ ((totalChronon < 0 && totalChronon % dayTime != 0) ? -1 : 0);
break;
default:
- throw new AlgebricksException(getIdentifier().getName()
- + ": expecting YEARMONTHDURATION/DAYTIMEDURATION for the thrid argument but got "
- + type2);
+ throw new TypeMismatchException(getIdentifier().getName(), 2, bytes2[offset2],
+ ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG);
}
switch (type0) {
@@ -207,8 +209,8 @@
break;
case TIME:
if (yearMonth != 0) {
- throw new AlgebricksException(getIdentifier().getName()
- + ": cannot create year-month bin for a time value");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
binStartChronon = DurationArithmeticOperations.addDuration(chrononToStart,
yearMonth * (int) binIndex, dayTime * binIndex, true);
@@ -222,15 +224,12 @@
yearMonth * ((int) binIndex + 1), dayTime * (binIndex + 1), false);
break;
default:
- throw new AlgebricksException(getIdentifier().getName()
- + ": the first argument should be DATE/TIME/DATETIME/NULL but got " + type0);
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_TIME_TYPE_TAG,
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
- try {
- aInterval.setValue(binStartChronon, binEndChronon, type0.serialize());
- intervalSerde.serialize(aInterval, out);
- } catch (HyracksDataException ex) {
- throw new AlgebricksException(ex);
- }
+ aInterval.setValue(binStartChronon, binEndChronon, type0.serialize());
+ intervalSerde.serialize(aInterval, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java
index ffca20751..5c49eed 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java
@@ -24,14 +24,13 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -55,14 +54,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -77,25 +74,17 @@
AMutableInt64 aInt64 = new AMutableInt64(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
-
byte[] bytes = argPtr0.getByteArray();
int offset = argPtr0.getStartOffset();
-
- try {
- if (bytes[offset] != ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects NULL/DAY-TIME-DURATION, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]));
- }
-
- aInt64.setValue(ADayTimeDurationSerializerDeserializer.getDayTime(bytes, offset + 1));
- int64Serde.serialize(aInt64, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ if (bytes[offset] != ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+ ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG);
}
+ aInt64.setValue(ADayTimeDurationSerializerDeserializer.getDayTime(bytes, offset + 1));
+ int64Serde.serialize(aInt64, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java
index 0d2b934..68b5f33 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java
@@ -24,14 +24,13 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -55,14 +54,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -77,25 +74,18 @@
AMutableInt64 aInt64 = new AMutableInt64(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();;
eval0.evaluate(tuple, argPtr0);
-
byte[] bytes = argPtr0.getByteArray();
int offset = argPtr0.getStartOffset();
- try {
- if (bytes[offset] != ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects NULL/YEAR-MONTH-DURATION, but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]));
- }
-
- aInt64.setValue(AYearMonthDurationSerializerDeserializer.getYearMonth(bytes, offset + 1));
- int64Serde.serialize(aInt64, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ if (bytes[offset] != ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+ ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG);
}
+ aInt64.setValue(AYearMonthDurationSerializerDeserializer.getYearMonth(bytes, offset + 1));
+ int64Serde.serialize(aInt64, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapBinsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapBinsDescriptor.java
index 15229e2..79fc90b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapBinsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapBinsDescriptor.java
@@ -36,18 +36,22 @@
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.typecomputer.impl.ADateTypeComputer;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.OverflowException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.runtime.exceptions.UnderflowException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -65,14 +69,13 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -99,7 +102,7 @@
private final GregorianCalendarSystem gregCalSys = GregorianCalendarSystem.getInstance();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -122,8 +125,8 @@
intervalStart = intervalStart * GregorianCalendarSystem.CHRONON_OF_DAY;
}
} else {
- throw new AlgebricksException(getIdentifier().getName()
- + ": the first argument should be INTERVAL/NULL/MISSING but got " + type0);
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
// get the anchor instance time
@@ -131,8 +134,7 @@
int offset1 = argPtr1.getStartOffset();
ATypeTag type1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
if (intervalTypeTag != bytes1[offset1]) {
- throw new AlgebricksException(getIdentifier().getName() + ": expecting compatible type to "
- + type0 + "(" + intervalTypeTag + ") for the second argument but got " + type1);
+ throw new IncompatibleTypeException(getIdentifier(), intervalTypeTag, bytes1[offset1]);
}
long anchorTime;
@@ -148,9 +150,9 @@
anchorTime = ADateTimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
break;
default:
- throw new AlgebricksException(
- getIdentifier().getName() + ": expecting compatible type to " + type0 + "("
- + intervalTypeTag + ") for the second argument but got " + type1);
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_TIME_TYPE_TAG,
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
byte[] bytes2 = argPtr2.getByteArray();
@@ -176,13 +178,11 @@
+ ((totalMonths < 0 && totalMonths % yearMonth != 0) ? -1 : 0);
if (firstBinIndex > Integer.MAX_VALUE) {
- throw new AlgebricksException(
- getIdentifier().getName() + ": Overflowing time value to be binned!");
+ throw new OverflowException(getIdentifier());
}
if (firstBinIndex < Integer.MIN_VALUE) {
- throw new AlgebricksException(
- getIdentifier().getName() + ": Underflowing time value to be binned!");
+ throw new UnderflowException(getIdentifier());
}
break;
@@ -195,17 +195,15 @@
+ ((totalChronon < 0 && totalChronon % dayTime != 0) ? -1 : 0);
break;
default:
- throw new AlgebricksException(getIdentifier().getName()
- + ": expecting YEARMONTHDURATION/DAYTIMEDURATION for the thrid argument but got "
- + type2);
+ throw new TypeMismatchException(getIdentifier(), 2, bytes2[offset2],
+ ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG);
}
long binStartChronon;
long binEndChronon;
int binOffset;
-
listBuilder.reset(intListType);
-
try {
if (intervalTypeTag == ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
@@ -235,8 +233,8 @@
} else if (intervalTypeTag == ATypeTag.SERIALIZED_TIME_TYPE_TAG) {
if (yearMonth != 0) {
- throw new AlgebricksException(getIdentifier().getName()
- + ": cannot create year-month bin for a time value");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
binOffset = 0;
@@ -250,8 +248,8 @@
if (binStartChronon < 0 || binStartChronon >= GregorianCalendarSystem.CHRONON_OF_DAY) {
// avoid the case where a time bin is before 00:00:00 or no early than 24:00:00
- throw new AlgebricksException(getIdentifier().getName()
- + ": reaches a bin with the end earlier than the start; probably the window is beyond the time scope. Maybe use DATETIME?");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
while (!((binStartChronon < intervalStart && binEndChronon <= intervalStart)
@@ -274,8 +272,8 @@
}
if (binEndChronon < binStartChronon) {
- throw new AlgebricksException(getIdentifier().getName()
- + ": reaches a bin with the end earlier than the start; probably the window is beyond the time scope. Maybe use DATETIME?");
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
}
}
} else if (intervalTypeTag == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
@@ -294,12 +292,13 @@
binOffset++;
} while (binEndChronon < intervalEnd);
} else {
- throw new AlgebricksException(getIdentifier().getName()
- + ": the first argument should be DATE/TIME/DATETIME/NULL but got " + type0);
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_TIME_TYPE_TAG,
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
listBuilder.write(out, true);
} catch (IOException e1) {
- throw new AlgebricksException(e1.getMessage());
+ throw new HyracksDataException(e1);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java
index 1810b48..9aff8c9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java
@@ -25,16 +25,16 @@
import org.apache.asterix.om.base.AMutableDate;
import org.apache.asterix.om.base.temporal.AsterixTemporalTypeParseException;
import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
-import org.apache.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -64,14 +64,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -89,7 +87,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -101,54 +99,49 @@
int offset1 = argPtr1.getStartOffset();
int len1 = argPtr1.getLength();
- try {
- if (bytes0[offset0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG
- || bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(getIdentifier().getName()
- + ": expects two strings but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]) + ")");
- }
- utf8Ptr.set(bytes0, offset0 + 1, len0 - 1);
- int start0 = utf8Ptr.getCharStartOffset();
- int length0 = utf8Ptr.getUTF8Length();
-
- utf8Ptr.set(bytes1, offset1 + 1, len1 - 1);
- int start1 = utf8Ptr.getCharStartOffset();
- int length1 = utf8Ptr.getUTF8Length();
- long chronon = 0;
-
- int formatStart = start1;
- int formatLength = 0;
- boolean processSuccessfully = false;
- while (!processSuccessfully && formatStart < start1 + length1) {
- // search for "|"
- formatLength = 0;
- for (; formatStart + formatLength < start1 + length1; formatLength++) {
- if (argPtr1.getByteArray()[formatStart + formatLength] == '|') {
- break;
- }
- }
- try {
- chronon = DT_UTILS.parseDateTime(bytes0, start0, length0, bytes1, formatStart,
- formatLength, DateTimeParseMode.DATE_ONLY);
- } catch (AsterixTemporalTypeParseException ex) {
- formatStart += formatLength + 1;
- continue;
- }
- processSuccessfully = true;
- }
-
- if (!processSuccessfully) {
- throw new HyracksDataException(
- "parse-date: Failed to match with any given format string!");
- }
-
- aDate.setValue((int) (chronon / GregorianCalendarSystem.CHRONON_OF_DAY));
- dateSerde.serialize(aDate, out);
- } catch (HyracksDataException ex) {
- throw new AlgebricksException(ex);
+ if (bytes0[offset0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+
+ utf8Ptr.set(bytes0, offset0 + 1, len0 - 1);
+ int start0 = utf8Ptr.getCharStartOffset();
+ int length0 = utf8Ptr.getUTF8Length();
+
+ utf8Ptr.set(bytes1, offset1 + 1, len1 - 1);
+ int start1 = utf8Ptr.getCharStartOffset();
+ int length1 = utf8Ptr.getUTF8Length();
+ long chronon = 0;
+
+ int formatStart = start1;
+ int formatLength;
+ boolean processSuccessfully = false;
+ while (!processSuccessfully && formatStart < start1 + length1) {
+ // search for "|"
+ formatLength = 0;
+ for (; formatStart + formatLength < start1 + length1; formatLength++) {
+ if (argPtr1.getByteArray()[formatStart + formatLength] == '|') {
+ break;
+ }
+ }
+ try {
+ chronon = DT_UTILS.parseDateTime(bytes0, start0, length0, bytes1, formatStart,
+ formatLength, DateTimeParseMode.DATE_ONLY);
+ } catch (AsterixTemporalTypeParseException ex) {
+ formatStart += formatLength + 1;
+ continue;
+ }
+ processSuccessfully = true;
+ }
+ if (!processSuccessfully) {
+ throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_DATE_TYPE_TAG);
+ }
+ aDate.setValue((int) (chronon / GregorianCalendarSystem.CHRONON_OF_DAY));
+ dateSerde.serialize(aDate, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java
index 9717f8e..24f62ba 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java
@@ -7,7 +7,7 @@
* "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
+ *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
@@ -31,9 +31,9 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -59,14 +59,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -84,7 +82,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -96,53 +94,50 @@
int offset1 = argPtr1.getStartOffset();
int len1 = argPtr1.getLength();
- try {
- if (bytes0[offset0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG
- || bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(getIdentifier().getName()
- + ": expects two strings but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]) + ")");
- }
- utf8Ptr.set(bytes0, offset0 + 1, len0 - 1);
- int start0 = utf8Ptr.getCharStartOffset();
- int length0 = utf8Ptr.getUTF8Length();
-
- utf8Ptr.set(bytes1, offset1 + 1, len1 - 1);
- int start1 = utf8Ptr.getCharStartOffset();
- int length1 = utf8Ptr.getUTF8Length();
- long chronon = 0;
-
- int formatStart = start1;
- int formatLength = 0;
- boolean processSuccessfully = false;
- while (!processSuccessfully && formatStart < start1 + length1) {
- // search for "|"
- formatLength = 0;
- for (; formatStart + formatLength < start1 + length1; formatLength++) {
- if (bytes1[formatStart + formatLength] == '|') {
- break;
- }
- }
- try {
- chronon = DT_UTILS.parseDateTime(bytes0, start0, length0, bytes1, formatStart,
- formatLength, DateTimeParseMode.DATETIME);
- } catch (AsterixTemporalTypeParseException ex) {
- formatStart += formatLength + 1;
- continue;
- }
- processSuccessfully = true;
- }
-
- if (!processSuccessfully) {
- throw new HyracksDataException(
- "parse-datetime: Failed to match with any given format string!");
- }
- aDateTime.setValue(chronon);
- datetimeSerde.serialize(aDateTime, out);
- } catch (HyracksDataException ex) {
- throw new AlgebricksException(ex);
+ if (bytes0[offset0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+ utf8Ptr.set(bytes0, offset0 + 1, len0 - 1);
+ int start0 = utf8Ptr.getCharStartOffset();
+ int length0 = utf8Ptr.getUTF8Length();
+
+ utf8Ptr.set(bytes1, offset1 + 1, len1 - 1);
+ int start1 = utf8Ptr.getCharStartOffset();
+ int length1 = utf8Ptr.getUTF8Length();
+ long chronon = 0;
+
+ int formatStart = start1;
+ int formatLength;
+ boolean processSuccessfully = false;
+ while (!processSuccessfully && formatStart < start1 + length1) {
+ // search for "|"
+ formatLength = 0;
+ for (; formatStart + formatLength < start1 + length1; formatLength++) {
+ if (bytes1[formatStart + formatLength] == '|') {
+ break;
+ }
+ }
+ try {
+ chronon = DT_UTILS.parseDateTime(bytes0, start0, length0, bytes1, formatStart,
+ formatLength, DateTimeParseMode.DATETIME);
+ } catch (AsterixTemporalTypeParseException ex) {
+ formatStart += formatLength + 1;
+ continue;
+ }
+ processSuccessfully = true;
+ }
+
+ if (!processSuccessfully) {
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
+ }
+ aDateTime.setValue(chronon);
+ datetimeSerde.serialize(aDateTime, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java
index fb4bc55..3b07c98 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java
@@ -31,9 +31,9 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -60,14 +60,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -85,7 +83,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -97,54 +95,50 @@
int offset1 = argPtr1.getStartOffset();
int len1 = argPtr1.getLength();
- try {
- if (bytes0[offset0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG
- || bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(getIdentifier().getName()
- + ": expects two strings but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]) + ")");
- }
- utf8Ptr.set(bytes0, offset0 + 1, len0 - 1);
- int start0 = utf8Ptr.getCharStartOffset();
- int length0 = utf8Ptr.getUTF8Length();
-
- utf8Ptr.set(bytes1, offset1 + 1, len1 - 1);
- int start1 = utf8Ptr.getCharStartOffset();
- int length1 = utf8Ptr.getUTF8Length();
- long chronon = 0;
-
- int formatStart = start1;
- int formatLength = 0;
- boolean processSuccessfully = false;
- while (!processSuccessfully && formatStart < start1 + length1) {
- // search for "|"
- formatLength = 0;
- for (; formatStart + formatLength < start1 + length1; formatLength++) {
- if (bytes1[formatStart + formatLength] == '|') {
- break;
- }
- }
- try {
- chronon = DT_UTILS.parseDateTime(bytes0, start0, length0, bytes1, formatStart,
- formatLength, DateTimeParseMode.TIME_ONLY);
- } catch (AsterixTemporalTypeParseException ex) {
- formatStart += formatLength + 1;
- continue;
- }
- processSuccessfully = true;
- }
-
- if (!processSuccessfully) {
- throw new HyracksDataException(
- "parse-date: Failed to match with any given format string!");
- }
-
- aTime.setValue((int) chronon);
- timeSerde.serialize(aTime, out);
- } catch (HyracksDataException ex) {
- throw new AlgebricksException(ex);
+ if (bytes0[offset0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+ utf8Ptr.set(bytes0, offset0 + 1, len0 - 1);
+ int start0 = utf8Ptr.getCharStartOffset();
+ int length0 = utf8Ptr.getUTF8Length();
+
+ utf8Ptr.set(bytes1, offset1 + 1, len1 - 1);
+ int start1 = utf8Ptr.getCharStartOffset();
+ int length1 = utf8Ptr.getUTF8Length();
+ long chronon = 0;
+
+ int formatStart = start1;
+ int formatLength;
+ boolean processSuccessfully = false;
+ while (!processSuccessfully && formatStart < start1 + length1) {
+ // search for "|"
+ formatLength = 0;
+ for (; formatStart + formatLength < start1 + length1; formatLength++) {
+ if (bytes1[formatStart + formatLength] == '|') {
+ break;
+ }
+ }
+ try {
+ chronon = DT_UTILS.parseDateTime(bytes0, start0, length0, bytes1, formatStart,
+ formatLength, DateTimeParseMode.TIME_ONLY);
+ } catch (AsterixTemporalTypeParseException ex) {
+ formatStart += formatLength + 1;
+ continue;
+ }
+ processSuccessfully = true;
+ }
+
+ if (!processSuccessfully) {
+ throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+ }
+
+ aTime.setValue((int) chronon);
+ timeSerde.serialize(aTime, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java
index b979786..cec78cc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java
@@ -23,19 +23,19 @@
import org.apache.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
-import org.apache.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -57,14 +57,13 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -78,7 +77,7 @@
private final UTF8StringWriter utf8Writer = new UTF8StringWriter();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -89,14 +88,14 @@
int offset1 = argPtr1.getStartOffset();
try {
- if (bytes0[offset0] != ATypeTag.SERIALIZED_DATE_TYPE_TAG
- || bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(getIdentifier().getName()
- + ": expects (DATE, STRING) but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]) + ")");
+ if (bytes0[offset0] != ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG);
}
-
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
long chronon = ADateSerializerDeserializer.getChronon(bytes0, offset0 + 1)
* GregorianCalendarSystem.CHRONON_OF_DAY;
int formatLength = UTF8StringUtil.getUTFLength(bytes1, offset1 + 1);
@@ -108,7 +107,7 @@
out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
utf8Writer.writeUTF8(sbder.toString(), out);
} catch (IOException ex) {
- throw new AlgebricksException(ex);
+ throw new HyracksDataException(ex);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java
index 780d09a..cca9a93 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java
@@ -24,17 +24,17 @@
import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
import org.apache.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -56,14 +56,13 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -77,7 +76,7 @@
private UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -89,12 +88,13 @@
int len1 = argPtr1.getLength();
try {
- if (bytes0[offset0] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG
- || bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(getIdentifier().getName()
- + ": expects (DATETIME, STRING) but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]) + ")");
+ if (bytes0[offset0] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
+ }
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
long chronon = ADateTimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
utf8Ptr.set(bytes1, offset1 + 1, len1 - 1);
@@ -106,7 +106,7 @@
out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
utf8Writer.writeUTF8(sbder.toString(), out);
} catch (IOException ex) {
- throw new AlgebricksException(ex);
+ throw new HyracksDataException(ex);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java
index 576de35..e6201c5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java
@@ -24,17 +24,17 @@
import org.apache.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
import org.apache.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -56,16 +56,13 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
-
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput out = resultStorage.getDataOutput();
private IPointable argPtr0 = new VoidPointable();
@@ -78,7 +75,7 @@
private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -90,12 +87,13 @@
int len1 = argPtr1.getLength();
try {
- if (bytes0[offset0] != ATypeTag.SERIALIZED_TIME_TYPE_TAG
- || bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- throw new AlgebricksException(getIdentifier().getName()
- + ": expects (TIME, STRING) but got ("
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + ", "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]) + ")");
+ if (bytes0[offset0] != ATypeTag.SERIALIZED_TIME_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+ }
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
long chronon = ATimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
@@ -108,7 +106,7 @@
out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
writer.writeUTF8(sbder.toString(), out);
} catch (IOException ex) {
- throw new AlgebricksException(ex);
+ throw new HyracksDataException(ex);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromDatetimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromDatetimeDescriptor.java
index 6846ef8..b7637a6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromDatetimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromDatetimeDescriptor.java
@@ -25,14 +25,13 @@
import org.apache.asterix.om.base.AMutableTime;
import org.apache.asterix.om.base.ATime;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -61,14 +60,13 @@
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
*/
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -83,29 +81,24 @@
private AMutableTime aTime = new AMutableTime(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int offset = argPtr.getStartOffset();
- try {
- if (bytes[offset] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
- throw new AlgebricksException(
- FID.getName() + ": expects input type DATETIME/NULL but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]));
- }
- long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
- int timeChronon = (int) (datetimeChronon % GregorianCalendarSystem.CHRONON_OF_DAY);
- if (timeChronon < 0) {
- timeChronon += GregorianCalendarSystem.CHRONON_OF_DAY;
- }
- aTime.setValue(timeChronon);
- timeSerde.serialize(aTime, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ if (bytes[offset] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
+ long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
+ int timeChronon = (int) (datetimeChronon % GregorianCalendarSystem.CHRONON_OF_DAY);
+ if (timeChronon < 0) {
+ timeChronon += GregorianCalendarSystem.CHRONON_OF_DAY;
+ }
+ aTime.setValue(timeChronon);
+ timeSerde.serialize(aTime, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromUnixTimeInMsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromUnixTimeInMsDescriptor.java
index e46241d..6420a75 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromUnixTimeInMsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromUnixTimeInMsDescriptor.java
@@ -29,7 +29,6 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -56,14 +55,12 @@
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
*/
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -79,16 +76,12 @@
private AMutableTime aTime = new AMutableTime(0);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval.evaluate(tuple, argPtr);
- try {
- aTime.setValue(
- ATypeHierarchy.getIntegerValue(argPtr.getByteArray(), argPtr.getStartOffset()));
- timeSerde.serialize(aTime, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
- }
+ aTime.setValue(ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0,
+ argPtr.getByteArray(), argPtr.getStartOffset()));
+ timeSerde.serialize(aTime, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDateInDaysDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDateInDaysDescriptor.java
index 7268302..db8161c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDateInDaysDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDateInDaysDescriptor.java
@@ -24,14 +24,13 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -55,14 +54,13 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -77,24 +75,19 @@
.getSerializerDeserializer(BuiltinType.AINT64);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int offset = argPtr.getStartOffset();
- try {
- if (bytes[offset] != ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
- throw new AlgebricksException(
- getIdentifier().getName() + ": expects input type DATE/NULL but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]));
- }
- long dateChronon = ADateSerializerDeserializer.getChronon(bytes, offset + 1);
- aInt64.setValue(dateChronon);
- int64Serde.serialize(aInt64, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ if (bytes[offset] != ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG);
}
+ long dateChronon = ADateSerializerDeserializer.getChronon(bytes, offset + 1);
+ aInt64.setValue(dateChronon);
+ int64Serde.serialize(aInt64, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsDescriptor.java
index fe76f32..2f24861 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsDescriptor.java
@@ -24,14 +24,13 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -54,14 +53,12 @@
};
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -76,24 +73,19 @@
.getSerializerDeserializer(BuiltinType.AINT64);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int offset = argPtr.getStartOffset();
- try {
- if (bytes[offset] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
- throw new AlgebricksException(
- getIdentifier().getName() + ": expects input type DATETIME/NULL but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]));
- }
- long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
- aInt64.setValue(datetimeChronon);
- int64Serde.serialize(aInt64, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ if (bytes[offset] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
+ long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
+ aInt64.setValue(datetimeChronon);
+ int64Serde.serialize(aInt64, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsDescriptor.java
index 162f002..3f0757b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsDescriptor.java
@@ -18,20 +18,21 @@
*/
package org.apache.asterix.runtime.evaluators.functions.temporal;
+import static org.apache.asterix.om.types.ATypeTag.SERIALIZED_DATETIME_TYPE_TAG;
+
import java.io.DataOutput;
import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -58,14 +59,12 @@
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
*/
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -80,24 +79,19 @@
.getSerializerDeserializer(BuiltinType.AINT64);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int offset = argPtr.getStartOffset();
- try {
- if (bytes[offset] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
- throw new AlgebricksException(
- getIdentifier().getName() + ": expects input type DATETIME/NULL but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]));
- }
- long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
- aInt64.setValue(datetimeChronon / 1000L);
- int64Serde.serialize(aInt64, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ if (bytes[offset] != SERIALIZED_DATETIME_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
+ long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
+ aInt64.setValue(datetimeChronon / 1000L);
+ int64Serde.serialize(aInt64, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromTimeInMsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromTimeInMsDescriptor.java
index 1beb772..7213479 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromTimeInMsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromTimeInMsDescriptor.java
@@ -24,14 +24,13 @@
import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -56,14 +55,12 @@
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
*/
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -78,24 +75,19 @@
.getSerializerDeserializer(BuiltinType.AINT64);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval.evaluate(tuple, argPtr);
byte[] bytes = argPtr.getByteArray();
int offset = argPtr.getStartOffset();
- try {
- if (bytes[offset] != ATypeTag.SERIALIZED_TIME_TYPE_TAG) {
- throw new AlgebricksException(
- getIdentifier().getName() + ": expects input type TIME/NULL but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]));
- }
- long timeChronon = ATimeSerializerDeserializer.getChronon(bytes, offset + 1);
- aInt64.setValue(timeChronon);
- int64Serde.serialize(aInt64, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ if (bytes[offset] != ATypeTag.SERIALIZED_TIME_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+ ATypeTag.SERIALIZED_TIME_TYPE_TAG);
}
+ long timeChronon = ATimeSerializerDeserializer.getChronon(bytes, offset + 1);
+ aInt64.setValue(timeChronon);
+ int64Serde.serialize(aInt64, out);
result.set(resultStorage);
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationComparatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationComparatorDescriptor.java
index 10b8116..78c40c5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationComparatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationComparatorDescriptor.java
@@ -26,9 +26,9 @@
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -51,14 +51,13 @@
}
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -73,7 +72,7 @@
.getSerializerDeserializer(BuiltinType.ABOOLEAN);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
eval0.evaluate(tuple, argPtr0);
eval1.evaluate(tuple, argPtr1);
@@ -83,29 +82,26 @@
byte[] bytes1 = argPtr1.getByteArray();
int offset1 = argPtr1.getStartOffset();
- try {
- if (bytes0[offset0] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG
- || bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
- throw new AlgebricksException(getIdentifier().getName()
- + ": expects type NULL/DURATION, NULL/DURATION but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + " and "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]));
- }
+ if (bytes0[offset0] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
+ }
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
+ }
- if ((ADurationSerializerDeserializer.getDayTime(bytes0, offset0 + 1) != 0)
- || (ADurationSerializerDeserializer.getDayTime(bytes1, offset1 + 1) != 0)) {
- throw new AlgebricksException(
- getIdentifier().getName() + ": only year-month durations are allowed.");
- }
+ if ((ADurationSerializerDeserializer.getDayTime(bytes0, offset0 + 1) != 0)
+ || (ADurationSerializerDeserializer.getDayTime(bytes1, offset1 + 1) != 0)) {
+ throw new InvalidDataFormatException(getIdentifier(),
+ ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG);
+ }
- if (ADurationSerializerDeserializer.getYearMonth(bytes0,
- offset0 + 1) > ADurationSerializerDeserializer.getYearMonth(bytes1, offset1 + 1)) {
- boolSerde.serialize(isGreaterThan ? ABoolean.TRUE : ABoolean.FALSE, out);
- } else {
- boolSerde.serialize(isGreaterThan ? ABoolean.FALSE : ABoolean.TRUE, out);
- }
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ if (ADurationSerializerDeserializer.getYearMonth(bytes0,
+ offset0 + 1) > ADurationSerializerDeserializer.getYearMonth(bytes1, offset1 + 1)) {
+ boolSerde.serialize(isGreaterThan ? ABoolean.TRUE : ABoolean.FALSE, out);
+ } else {
+ boolSerde.serialize(isGreaterThan ? ABoolean.FALSE : ABoolean.TRUE, out);
}
result.set(resultStorage);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/visitors/DeepEqualityVisitor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/visitors/DeepEqualityVisitor.java
index cd2718e..3cf5ec9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/visitors/DeepEqualityVisitor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/visitors/DeepEqualityVisitor.java
@@ -96,7 +96,8 @@
if (domain == Domain.NUMERIC) {
int s1 = pointable.getStartOffset();
int s2 = arg.first.getStartOffset();
- arg.second = (ATypeHierarchy.getDoubleValue(b1, s1) == ATypeHierarchy.getDoubleValue(b2, s2));
+ arg.second = Math.abs(ATypeHierarchy.getDoubleValue("deep-equal", 0, b1, s1)
+ - ATypeHierarchy.getDoubleValue("deep-equal", 1, b2, s2)) < 1E-10;
} else {
arg.second = false;
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/ExceptionUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/ExceptionUtil.java
new file mode 100644
index 0000000..cf9ae22
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/ExceptionUtil.java
@@ -0,0 +1,64 @@
+/*
+ * 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.asterix.runtime.exceptions;
+
+import org.apache.asterix.om.types.EnumDeserializer;
+
+public class ExceptionUtil {
+ private ExceptionUtil() {
+ }
+
+ static String toExpectedTypeString(byte... expectedTypeTags) {
+ StringBuilder expectedTypes = new StringBuilder();
+ int numCandidateTypes = expectedTypeTags.length;
+ for (int index = 0; index < numCandidateTypes; ++index) {
+ if (index > 0) {
+ if (index == numCandidateTypes - 1) {
+ expectedTypes.append(" or ");
+ } else {
+ expectedTypes.append(", ");
+ }
+ }
+ expectedTypes.append(EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(expectedTypeTags[index]));
+ }
+ return expectedTypes.toString();
+ }
+
+ static String indexToPosition(int index) {
+ int i = index + 1;
+ switch (i % 100) {
+ case 11:
+ case 12:
+ case 13:
+ return i + "th";
+ default:
+ switch (i % 10) {
+ case 1:
+ return i + "st";
+ case 2:
+ return i + "nd";
+ case 3:
+ return i + "rd";
+ default:
+ return i + "th";
+ }
+ }
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/IncompatibleTypeException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/IncompatibleTypeException.java
new file mode 100644
index 0000000..0dad96c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/IncompatibleTypeException.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.asterix.runtime.exceptions;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class IncompatibleTypeException extends RuntimeDataException {
+
+ // Incompatible input parameters, e.g., "1.0" > 1.0
+ public IncompatibleTypeException(FunctionIdentifier fid, byte typeTagLeft, byte typeTagRight) {
+ super(ErrorCode.ERROR_TYPE_INCOMPATIBLE, fid.getName(),
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagLeft),
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagRight));
+ }
+
+ // Incompatible input parameters, e.g., "1.0" > 1.0
+ public IncompatibleTypeException(String functionName, byte typeTagLeft, byte typeTagRight) {
+ super(ErrorCode.ERROR_TYPE_INCOMPATIBLE, functionName,
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagLeft),
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagRight));
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java
new file mode 100644
index 0000000..0fdb1f0
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java
@@ -0,0 +1,43 @@
+/*
+ * 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.asterix.runtime.exceptions;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class InvalidDataFormatException extends RuntimeDataException {
+
+ public InvalidDataFormatException(FunctionIdentifier fid, byte expectedTypeTag) {
+ super(ErrorCode.ERROR_INVALID_FORMAT, fid.getName(),
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(expectedTypeTag));
+ }
+
+ public InvalidDataFormatException(FunctionIdentifier fid, String expectedType) {
+ super(ErrorCode.ERROR_INVALID_FORMAT, fid.getName(), expectedType);
+ }
+
+ public InvalidDataFormatException(FunctionIdentifier fid, Throwable cause, byte expectedTypeTag) {
+ super(ErrorCode.ERROR_INVALID_FORMAT, fid.getName(), cause, expectedTypeTag);
+ addSuppressed(cause);
+ }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/OverflowException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/OverflowException.java
new file mode 100644
index 0000000..968a54f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/OverflowException.java
@@ -0,0 +1,38 @@
+/*
+ * 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.asterix.runtime.exceptions;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class OverflowException extends RuntimeDataException {
+
+ // Overflow.
+ public OverflowException(FunctionIdentifier fid) {
+ super(ErrorCode.ERROR_OVERFLOW, fid.getName());
+ }
+
+ // Overflow.
+ public OverflowException(String functionName) {
+ super(ErrorCode.ERROR_OVERFLOW, functionName);
+ }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/TypeMismatchException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/TypeMismatchException.java
new file mode 100644
index 0000000..2a838a2
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/TypeMismatchException.java
@@ -0,0 +1,45 @@
+/*
+ * 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.asterix.runtime.exceptions;
+
+import static org.apache.asterix.runtime.exceptions.ExceptionUtil.indexToPosition;
+import static org.apache.asterix.runtime.exceptions.ExceptionUtil.toExpectedTypeString;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class TypeMismatchException extends RuntimeDataException {
+
+ // Parameter type mistmatch.
+ public TypeMismatchException(FunctionIdentifier fid, Integer i, byte actualTypeTag, byte... expectedTypeTags) {
+ super(ErrorCode.ERROR_TYPE_MISMATCH, fid.getName(), indexToPosition(i), toExpectedTypeString(expectedTypeTags),
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
+ }
+
+ // Parameter type mistmatch.
+ public TypeMismatchException(String functionName, Integer i, byte actualTypeTag, byte... expectedTypeTags) {
+ super(ErrorCode.ERROR_TYPE_MISMATCH, functionName, indexToPosition(i), toExpectedTypeString(expectedTypeTags),
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
+ }
+
+
+}
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
new file mode 100644
index 0000000..fede04f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnderflowException.java
@@ -0,0 +1,38 @@
+/*
+ * 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.asterix.runtime.exceptions;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class UnderflowException extends RuntimeDataException {
+
+ // Underflow.
+ public UnderflowException(FunctionIdentifier fid) {
+ super(ErrorCode.ERROR_OVERFLOW, fid.getName());
+ }
+
+ // Underflow.
+ public UnderflowException(String functionName) {
+ super(ErrorCode.ERROR_OVERFLOW, functionName);
+ }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedItemTypeException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedItemTypeException.java
new file mode 100644
index 0000000..79d9266
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedItemTypeException.java
@@ -0,0 +1,38 @@
+/*
+ * 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.asterix.runtime.exceptions;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class UnsupportedItemTypeException extends RuntimeDataException {
+
+ // Unsupported item type.
+ public UnsupportedItemTypeException(FunctionIdentifier fid, byte itemTypeTag) {
+ super(ErrorCode.ERROR_TYPE_ITEM, fid.getName(), EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(itemTypeTag));
+ }
+
+ // Unsupported item type.
+ public UnsupportedItemTypeException(String functionName, byte itemTypeTag) {
+ super(ErrorCode.ERROR_TYPE_ITEM, functionName, EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(itemTypeTag));
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedTypeException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedTypeException.java
new file mode 100644
index 0000000..6279fbc
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedTypeException.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.asterix.runtime.exceptions;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class UnsupportedTypeException extends RuntimeDataException {
+
+ // Unsupported input type.
+ public UnsupportedTypeException(FunctionIdentifier fid, byte actualTypeTag) {
+ super(ErrorCode.ERROR_TYPE_UNSUPPORTED, fid.getName(),
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
+ }
+
+ // Unsupported input type.
+ public UnsupportedTypeException(String funcName, byte actualTypeTag) {
+ super(ErrorCode.ERROR_TYPE_UNSUPPORTED, funcName,
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java
index 9cdfa56..b3a17aa 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.runningaggregates.base.AbstractRunningAggregateFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
@@ -49,17 +48,14 @@
};
@Override
- public IRunningAggregateEvaluatorFactory createRunningAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
-
+ public IRunningAggregateEvaluatorFactory createRunningAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) {
return new IRunningAggregateEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@SuppressWarnings("unchecked")
@Override
public IRunningAggregateEvaluator createRunningAggregateEvaluator(IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new IRunningAggregateEvaluator() {
@@ -70,20 +66,16 @@
private int cnt;
@Override
- public void step(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
- try {
- m.setValue(cnt);
- serde.serialize(m, resultStorage.getDataOutput());
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ m.setValue(cnt);
+ serde.serialize(m, resultStorage.getDataOutput());
result.set(resultStorage);
++cnt;
}
@Override
- public void init() throws AlgebricksException {
+ public void init() throws HyracksDataException {
cnt = 1;
}
};
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/RangeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/RangeDescriptor.java
index 2f3cd7e..6acb8d4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/RangeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/RangeDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.unnestingfunctions.base.AbstractUnnestingFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -57,17 +56,15 @@
}
@Override
- public IUnnestingEvaluatorFactory createUnnestingEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IUnnestingEvaluatorFactory createUnnestingEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IUnnestingEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IUnnestingEvaluator createUnnestingEvaluator(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new IUnnestingEvaluator() {
-
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@SuppressWarnings("rawtypes")
private ISerializerDeserializer serde = AqlSerializerDeserializerProvider.INSTANCE
@@ -80,35 +77,25 @@
private long max;
@Override
- public void init(IFrameTupleReference tuple) throws AlgebricksException {
+ public void init(IFrameTupleReference tuple) throws HyracksDataException {
eval0.evaluate(tuple, inputVal);
- try {
- current = ATypeHierarchy.getLongValue(inputVal.getByteArray(), inputVal.getStartOffset());
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ current = ATypeHierarchy.getLongValue(getIdentifier().getName(), 0, inputVal.getByteArray(),
+ inputVal.getStartOffset());
eval1.evaluate(tuple, inputVal);
- try {
- max = ATypeHierarchy.getLongValue(inputVal.getByteArray(), inputVal.getStartOffset());
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ max = ATypeHierarchy.getLongValue(getIdentifier().getName(), 1, inputVal.getByteArray(),
+ inputVal.getStartOffset());
}
@SuppressWarnings("unchecked")
@Override
- public boolean step(IPointable result) throws AlgebricksException {
+ public boolean step(IPointable result) throws HyracksDataException {
if (current > max) {
return false;
}
aInt64.setValue(current);
- try {
- resultStorage.reset();
- serde.serialize(aInt64, resultStorage.getDataOutput());
- result.set(resultStorage);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ resultStorage.reset();
+ serde.serialize(aInt64, resultStorage.getDataOutput());
+ result.set(resultStorage);
current++;
return true;
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
index 2e54d8e..33a35e2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
@@ -7,7 +7,7 @@
* "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
+ *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
@@ -22,20 +22,20 @@
import java.io.IOException;
import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.common.AsterixListAccessor;
import org.apache.asterix.runtime.unnestingfunctions.base.AbstractUnnestingFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -71,10 +71,8 @@
}
@Override
- public IUnnestingEvaluator createUnnestingEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
-
+ public IUnnestingEvaluator createUnnestingEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IUnnestingEvaluator() {
-
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final AsterixListAccessor listAccessor = new AsterixListAccessor();
private final IPointable inputVal = new VoidPointable();
@@ -83,25 +81,26 @@
private boolean metUnknown = false;
@Override
- public void init(IFrameTupleReference tuple) throws AlgebricksException {
- try {
- metUnknown = false;
- argEval.evaluate(tuple, inputVal);
- ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
- .deserialize(inputVal.getByteArray()[inputVal.getStartOffset()]);
- if (typeTag == ATypeTag.MISSING || typeTag == ATypeTag.NULL) {
- metUnknown = true;
- return;
- }
- listAccessor.reset(inputVal.getByteArray(), inputVal.getStartOffset());
- itemIndex = 0;
- } catch (AsterixException e) {
- throw new AlgebricksException(e);
+ public void init(IFrameTupleReference tuple) throws HyracksDataException {
+ metUnknown = false;
+ argEval.evaluate(tuple, inputVal);
+ byte typeTag = inputVal.getByteArray()[inputVal.getStartOffset()];
+ if (typeTag == ATypeTag.SERIALIZED_MISSING_TYPE_TAG
+ || typeTag == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
+ metUnknown = true;
+ return;
}
+ if (typeTag != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
+ && typeTag != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
+ throw new TypeMismatchException(AsterixBuiltinFunctions.SCAN_COLLECTION, 0, typeTag,
+ ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG, ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
+ }
+ listAccessor.reset(inputVal.getByteArray(), inputVal.getStartOffset());
+ itemIndex = 0;
}
@Override
- public boolean step(IPointable result) throws AlgebricksException {
+ public boolean step(IPointable result) throws HyracksDataException {
try {
if (!metUnknown) {
if (itemIndex < listAccessor.size()) {
@@ -113,13 +112,12 @@
}
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
} catch (AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
return false;
}
-
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/SubsetCollectionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/SubsetCollectionDescriptor.java
index b6f8b0b..8cf3c1b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/SubsetCollectionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/SubsetCollectionDescriptor.java
@@ -7,7 +7,7 @@
* "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
+ *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
@@ -21,6 +21,8 @@
import java.io.IOException;
import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
@@ -31,13 +33,13 @@
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.om.util.NonTaggedFormatUtil;
import org.apache.asterix.runtime.unnestingfunctions.base.AbstractUnnestingFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -53,14 +55,12 @@
};
@Override
- public IUnnestingEvaluatorFactory createUnnestingEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IUnnestingEvaluatorFactory createUnnestingEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IUnnestingEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IUnnestingEvaluator createUnnestingEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
-
+ public IUnnestingEvaluator createUnnestingEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IUnnestingEvaluator() {
private IPointable inputVal = new VoidPointable();
private IScalarEvaluator evalList = args[0].createScalarEvaluator(ctx);
@@ -76,15 +76,16 @@
private boolean metUnknown = false;
@Override
- public void init(IFrameTupleReference tuple) throws AlgebricksException {
+ public void init(IFrameTupleReference tuple) throws HyracksDataException {
try {
evalStart.evaluate(tuple, inputVal);
- posStart = ATypeHierarchy.getIntegerValue(inputVal.getByteArray(),
+ posStart = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0,
+ inputVal.getByteArray(),
inputVal.getStartOffset());
evalLen.evaluate(tuple, inputVal);
- numItems = ATypeHierarchy.getIntegerValue(inputVal.getByteArray(),
- inputVal.getStartOffset());
+ numItems = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 1,
+ inputVal.getByteArray(), inputVal.getStartOffset());
evalList.evaluate(tuple, inputVal);
byte[] serList = inputVal.getByteArray();
@@ -100,8 +101,7 @@
if (typeTag != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
&& typeTag != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
- throw new AlgebricksException("Subset-collection is not defined for values of type"
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTag));
+ throw new RuntimeDataException(ErrorCode.ERROR_COERCION, getIdentifier());
}
if (typeTag == ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
numItemsMax = AOrderedListSerializerDeserializer.getNumberOfItems(serList, offset);
@@ -116,12 +116,12 @@
posCrt = posStart;
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
@Override
- public boolean step(IPointable result) throws AlgebricksException {
+ public boolean step(IPointable result) throws HyracksDataException {
if (!metUnknown && posCrt < posStart + numItems && posCrt < numItemsMax) {
resultStorage.reset();
byte[] serList = inputVal.getByteArray();
@@ -141,9 +141,9 @@
resultStorage.getDataOutput().write(serList, itemOffset,
itemLength + (!selfDescList ? 0 : 1));
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
} catch (AsterixException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
result.set(resultStorage);
++posCrt;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
index e9f9db0..574038b 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
@@ -26,9 +26,9 @@
import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionRuntimeProvider;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
import org.apache.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
@@ -178,7 +178,8 @@
}
@Override
- public synchronized ITuplePairComparator createTuplePairComparator(IHyracksTaskContext ctx) {
+ public synchronized ITuplePairComparator createTuplePairComparator(IHyracksTaskContext ctx)
+ throws HyracksDataException {
return new TuplePairEvaluator(ctx, cond, binaryBooleanInspectorFactory.createBinaryBooleanInspector(ctx));
}
}
@@ -186,7 +187,6 @@
public static class TuplePairEvaluator implements ITuplePairComparator {
private final IHyracksTaskContext ctx;
private IScalarEvaluator condEvaluator;
- private final IScalarEvaluatorFactory condFactory;
private final IPointable p;
private final CompositeFrameTupleReference compositeTupleRef;
private final FrameTupleReference leftRef;
@@ -194,9 +194,9 @@
private final IBinaryBooleanInspector binaryBooleanInspector;
public TuplePairEvaluator(IHyracksTaskContext ctx, IScalarEvaluatorFactory condFactory,
- IBinaryBooleanInspector binaryBooleanInspector) {
+ IBinaryBooleanInspector binaryBooleanInspector) throws HyracksDataException {
this.ctx = ctx;
- this.condFactory = condFactory;
+ this.condEvaluator = condFactory.createScalarEvaluator(ctx);
this.binaryBooleanInspector = binaryBooleanInspector;
this.leftRef = new FrameTupleReference();
this.p = VoidPointable.FACTORY.createPointable();
@@ -207,19 +207,8 @@
@Override
public int compare(IFrameTupleAccessor outerAccessor, int outerIndex, IFrameTupleAccessor innerAccessor,
int innerIndex) throws HyracksDataException {
- if (condEvaluator == null) {
- try {
- this.condEvaluator = condFactory.createScalarEvaluator(ctx);
- } catch (AlgebricksException ae) {
- throw new HyracksDataException(ae);
- }
- }
compositeTupleRef.reset(outerAccessor, outerIndex, innerAccessor, innerIndex);
- try {
- condEvaluator.evaluate(compositeTupleRef, p);
- } catch (AlgebricksException ae) {
- throw new HyracksDataException(ae);
- }
+ condEvaluator.evaluate(compositeTupleRef, p);
boolean result = binaryBooleanInspector.getBooleanValue(p.getByteArray(), p.getStartOffset(),
p.getLength());
if (result) {
diff --git a/hyracks-fullstack/algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/runtime/functions/IntegerEqFunctionEvaluatorFactory.java b/hyracks-fullstack/algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/runtime/functions/IntegerEqFunctionEvaluatorFactory.java
index a88b16b..7a83126 100644
--- a/hyracks-fullstack/algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/runtime/functions/IntegerEqFunctionEvaluatorFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/runtime/functions/IntegerEqFunctionEvaluatorFactory.java
@@ -18,10 +18,10 @@
*/
package org.apache.hyracks.algebricks.examples.piglet.runtime.functions;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.IntegerPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -40,7 +40,7 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private IPointable out1 = new VoidPointable();
private IPointable out2 = new VoidPointable();
@@ -49,7 +49,7 @@
private byte[] resultData = new byte[1];
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval1.evaluate(tuple, out1);
eval2.evaluate(tuple, out2);
int v1 = IntegerPointable.getInteger(out1.getByteArray(), out1.getStartOffset());
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/aggregators/TupleCountAggregateFunctionFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/aggregators/TupleCountAggregateFunctionFactory.java
index b5b25d0..f2c8948 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/aggregators/TupleCountAggregateFunctionFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/aggregators/TupleCountAggregateFunctionFactory.java
@@ -20,10 +20,10 @@
import java.io.IOException;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -33,35 +33,35 @@
private static final long serialVersionUID = 1L;
@Override
- public IAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
final ArrayBackedValueStorage abvs = new ArrayBackedValueStorage();
return new IAggregateEvaluator() {
int cnt;
@Override
- public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
++cnt;
}
@Override
- public void init() throws AlgebricksException {
+ public void init() throws HyracksDataException {
cnt = 0;
}
@Override
- public void finish(IPointable result) throws AlgebricksException {
+ public void finish(IPointable result) throws HyracksDataException {
try {
abvs.reset();
abvs.getDataOutput().writeInt(cnt);
result.set(abvs);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
@Override
- public void finishPartial(IPointable result) throws AlgebricksException {
+ public void finishPartial(IPointable result) throws HyracksDataException {
finish(result);
}
};
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/aggregators/TupleCountRunningAggregateFunctionFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/aggregators/TupleCountRunningAggregateFunctionFactory.java
index 01f1b88..64164f4 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/aggregators/TupleCountRunningAggregateFunctionFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/aggregators/TupleCountRunningAggregateFunctionFactory.java
@@ -20,10 +20,10 @@
import java.io.IOException;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -34,26 +34,26 @@
@Override
public IRunningAggregateEvaluator createRunningAggregateEvaluator(IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
final ArrayBackedValueStorage abvs = new ArrayBackedValueStorage();
return new IRunningAggregateEvaluator() {
int cnt;
@Override
- public void step(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
- ++cnt;
+ public void step(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
try {
+ ++cnt;
abvs.reset();
abvs.getDataOutput().writeInt(cnt);
result.set(abvs);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
@Override
- public void init() throws AlgebricksException {
+ public void init() throws HyracksDataException {
cnt = 0;
}
};
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IAggregateEvaluator.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IAggregateEvaluator.java
index 08aea9a..cced888 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IAggregateEvaluator.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IAggregateEvaluator.java
@@ -18,17 +18,17 @@
*/
package org.apache.hyracks.algebricks.runtime.base;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
public interface IAggregateEvaluator {
/** should be called each time a new aggregate value is computed */
- public void init() throws AlgebricksException;
+ public void init() throws HyracksDataException;
- public void step(IFrameTupleReference tuple) throws AlgebricksException;
+ public void step(IFrameTupleReference tuple) throws HyracksDataException;
- public void finish(IPointable result) throws AlgebricksException;
+ public void finish(IPointable result) throws HyracksDataException;
- public void finishPartial(IPointable result) throws AlgebricksException;
+ public void finishPartial(IPointable result) throws HyracksDataException;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IAggregateEvaluatorFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IAggregateEvaluatorFactory.java
index 8a61280..3f1cdd3 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IAggregateEvaluatorFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IAggregateEvaluatorFactory.java
@@ -20,9 +20,9 @@
import java.io.Serializable;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public interface IAggregateEvaluatorFactory extends Serializable {
- public IAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx) throws AlgebricksException;
+ public IAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx) throws HyracksDataException;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java
index 60fb517..de6cddd 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java
@@ -20,10 +20,9 @@
import java.io.Serializable;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
public interface IPushRuntimeFactory extends Serializable {
- public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws AlgebricksException, HyracksDataException;
+ public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IRunningAggregateEvaluator.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IRunningAggregateEvaluator.java
index 174dd7d..af51683 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IRunningAggregateEvaluator.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IRunningAggregateEvaluator.java
@@ -18,12 +18,12 @@
*/
package org.apache.hyracks.algebricks.runtime.base;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
public interface IRunningAggregateEvaluator {
- public void init() throws AlgebricksException;
+ public void init() throws HyracksDataException;
- public void step(IFrameTupleReference tuple, IPointable result) throws AlgebricksException;
+ public void step(IFrameTupleReference tuple, IPointable result) throws HyracksDataException;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IRunningAggregateEvaluatorFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IRunningAggregateEvaluatorFactory.java
index c71b41d..c85a051 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IRunningAggregateEvaluatorFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IRunningAggregateEvaluatorFactory.java
@@ -20,10 +20,10 @@
import java.io.Serializable;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public interface IRunningAggregateEvaluatorFactory extends Serializable {
public IRunningAggregateEvaluator createRunningAggregateEvaluator(IHyracksTaskContext ctx)
- throws AlgebricksException;
+ throws HyracksDataException;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IScalarEvaluator.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IScalarEvaluator.java
index ab22819..f5ef9c3 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IScalarEvaluator.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IScalarEvaluator.java
@@ -18,10 +18,10 @@
*/
package org.apache.hyracks.algebricks.runtime.base;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
public interface IScalarEvaluator {
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException;
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IScalarEvaluatorFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IScalarEvaluatorFactory.java
index 3a102db..d1508ee 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IScalarEvaluatorFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IScalarEvaluatorFactory.java
@@ -20,9 +20,9 @@
import java.io.Serializable;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public interface IScalarEvaluatorFactory extends Serializable {
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException;
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/ISerializedAggregateEvaluator.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/ISerializedAggregateEvaluator.java
index 473afbc..f40c1b9 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/ISerializedAggregateEvaluator.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/ISerializedAggregateEvaluator.java
@@ -21,6 +21,7 @@
import java.io.DataOutput;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
public interface ISerializedAggregateEvaluator {
@@ -30,16 +31,16 @@
* @param state
* @throws AlgebricksException
*/
- public void init(DataOutput state) throws AlgebricksException;
+ public void init(DataOutput state) throws HyracksDataException;
/**
* update the internal state
*
- * @param tuple
* @param state
+ * @param tuple
* @throws AlgebricksException
*/
- public void step(IFrameTupleReference tuple, byte[] data, int start, int len) throws AlgebricksException;
+ public void step(IFrameTupleReference tuple, byte[] data, int start, int len) throws HyracksDataException;
/**
* output the state to result
@@ -48,7 +49,7 @@
* @param result
* @throws AlgebricksException
*/
- public void finish(byte[] data, int start, int len, DataOutput result) throws AlgebricksException;
+ public void finish(byte[] data, int start, int len, DataOutput result) throws HyracksDataException;
/**
* output the partial state to partial result
@@ -57,5 +58,5 @@
* @param partialResult
* @throws AlgebricksException
*/
- public void finishPartial(byte[] data, int start, int len, DataOutput partialResult) throws AlgebricksException;
+ public void finishPartial(byte[] data, int start, int len, DataOutput partialResult) throws HyracksDataException;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/ISerializedAggregateEvaluatorFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/ISerializedAggregateEvaluatorFactory.java
index 084f39d..238f6c7 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/ISerializedAggregateEvaluatorFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/ISerializedAggregateEvaluatorFactory.java
@@ -20,9 +20,9 @@
import java.io.Serializable;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public interface ISerializedAggregateEvaluatorFactory extends Serializable {
- public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx) throws AlgebricksException;
+ public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx) throws HyracksDataException;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IUnnestingEvaluator.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IUnnestingEvaluator.java
index 67aede4..ecc8dc2 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IUnnestingEvaluator.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IUnnestingEvaluator.java
@@ -18,13 +18,13 @@
*/
package org.apache.hyracks.algebricks.runtime.base;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
public interface IUnnestingEvaluator {
- public void init(IFrameTupleReference tuple) throws AlgebricksException;
+ public void init(IFrameTupleReference tuple) throws HyracksDataException;
- public boolean step(IPointable result) throws AlgebricksException;
+ public boolean step(IPointable result) throws HyracksDataException;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IUnnestingEvaluatorFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IUnnestingEvaluatorFactory.java
index 53ca8b5..0e7b29e 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IUnnestingEvaluatorFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IUnnestingEvaluatorFactory.java
@@ -20,9 +20,9 @@
import java.io.Serializable;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public interface IUnnestingEvaluatorFactory extends Serializable {
- public IUnnestingEvaluator createUnnestingEvaluator(IHyracksTaskContext ctx) throws AlgebricksException;
+ public IUnnestingEvaluator createUnnestingEvaluator(IHyracksTaskContext ctx) throws HyracksDataException;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/ColumnAccessEvalFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/ColumnAccessEvalFactory.java
index 1b7753c..a2ae9c1 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/ColumnAccessEvalFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/ColumnAccessEvalFactory.java
@@ -18,10 +18,10 @@
*/
package org.apache.hyracks.algebricks.runtime.evaluators;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -41,11 +41,11 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
byte[] buffer = tuple.getFieldData(fieldIndex);
int start = tuple.getFieldStart(fieldIndex);
int length = tuple.getFieldLength(fieldIndex);
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/ConstantEvalFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/ConstantEvalFactory.java
index da7d42b..26cc5cb 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/ConstantEvalFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/ConstantEvalFactory.java
@@ -18,10 +18,10 @@
*/
package org.apache.hyracks.algebricks.runtime.evaluators;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -40,11 +40,11 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
result.set(value, 0, value.length);
}
};
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/TupleFieldEvaluatorFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/TupleFieldEvaluatorFactory.java
index f26d239..073a575 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/TupleFieldEvaluatorFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/TupleFieldEvaluatorFactory.java
@@ -18,10 +18,10 @@
*/
package org.apache.hyracks.algebricks.runtime.evaluators;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -35,10 +35,10 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
result.set(tuple.getFieldData(fieldIndex), tuple.getFieldStart(fieldIndex),
tuple.getFieldLength(fieldIndex));
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/AggregateRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/AggregateRuntimeFactory.java
index bafe8a7..42e5157 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/AggregateRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/AggregateRuntimeFactory.java
@@ -20,7 +20,6 @@
import java.nio.ByteBuffer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
@@ -61,9 +60,8 @@
@Override
public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
return new AbstractOneInputOneOutputOneFramePushRuntime() {
-
private IAggregateEvaluator[] aggregs = new IAggregateEvaluator[aggregFactories.length];
private IPointable result = VoidPointable.FACTORY.createPointable();
private ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(aggregs.length);
@@ -73,19 +71,15 @@
@Override
public void open() throws HyracksDataException {
- try {
- if (first) {
- first = false;
- initAccessAppendRef(ctx);
- for (int i = 0; i < aggregFactories.length; i++) {
- aggregs[i] = aggregFactories[i].createAggregateEvaluator(ctx);
- }
- }
+ if (first) {
+ first = false;
+ initAccessAppendRef(ctx);
for (int i = 0; i < aggregFactories.length; i++) {
- aggregs[i].init();
+ aggregs[i] = aggregFactories[i].createAggregateEvaluator(ctx);
}
- } catch (AlgebricksException e) {
- throw new HyracksDataException(e);
+ }
+ for (int i = 0; i < aggregFactories.length; i++) {
+ aggregs[i].init();
}
isOpen = true;
writer.open();
@@ -117,22 +111,14 @@
private void computeAggregate() throws HyracksDataException {
tupleBuilder.reset();
for (int f = 0; f < aggregs.length; f++) {
- try {
- aggregs[f].finish(result);
- } catch (AlgebricksException e) {
- throw new HyracksDataException(e);
- }
+ aggregs[f].finish(result);
tupleBuilder.addField(result.getByteArray(), result.getStartOffset(), result.getLength());
}
}
private void processTuple(FrameTupleReference tupleRef) throws HyracksDataException {
for (int f = 0; f < aggregs.length; f++) {
- try {
- aggregs[f].step(tupleRef);
- } catch (AlgebricksException e) {
- throw new HyracksDataException(e);
- }
+ aggregs[f].step(tupleRef);
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
index 688e819..b397f23 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
@@ -20,7 +20,6 @@
import java.nio.ByteBuffer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
@@ -57,11 +56,7 @@
final AggregatorOutput outputWriter = new AggregatorOutput(subplans, keyFieldIdx.length, decorFieldIdx.length);
final NestedTupleSourceRuntime[] pipelines = new NestedTupleSourceRuntime[subplans.length];
for (int i = 0; i < subplans.length; i++) {
- try {
- pipelines[i] = (NestedTupleSourceRuntime) assemblePipeline(subplans[i], outputWriter, ctx);
- } catch (AlgebricksException e) {
- throw new HyracksDataException(e);
- }
+ pipelines[i] = (NestedTupleSourceRuntime) assemblePipeline(subplans[i], outputWriter, ctx);
}
return new IAggregatorDescriptor() {
@@ -144,7 +139,7 @@
}
private IFrameWriter assemblePipeline(AlgebricksPipeline subplan, IFrameWriter writer, IHyracksTaskContext ctx)
- throws AlgebricksException, HyracksDataException {
+ throws HyracksDataException {
// plug the operators
IFrameWriter start = writer;
IPushRuntimeFactory[] runtimeFactories = subplan.getRuntimeFactories();
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java
index 4b63155..52245e1 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java
@@ -20,7 +20,6 @@
import java.nio.ByteBuffer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
@@ -63,11 +62,7 @@
decorFieldIdx.length, writer);
final NestedTupleSourceRuntime[] pipelines = new NestedTupleSourceRuntime[subplans.length];
for (int i = 0; i < subplans.length; i++) {
- try {
pipelines[i] = (NestedTupleSourceRuntime) assemblePipeline(subplans[i], outputWriter, ctx);
- } catch (AlgebricksException e) {
- throw new HyracksDataException(e);
- }
}
final ArrayTupleBuilder gbyTb = outputWriter.getGroupByTupleBuilder();
@@ -140,7 +135,7 @@
}
private IFrameWriter assemblePipeline(AlgebricksPipeline subplan, IFrameWriter writer, IHyracksTaskContext ctx)
- throws AlgebricksException, HyracksDataException {
+ throws HyracksDataException {
// plug the operators
IFrameWriter start = writer;
IPushRuntimeFactory[] runtimeFactories = subplan.getRuntimeFactories();
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java
index 6700fb8..e4ac701 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java
@@ -20,7 +20,6 @@
import java.io.DataOutput;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
import org.apache.hyracks.api.comm.IFrameTupleAccessor;
@@ -67,30 +66,22 @@
DataOutput output = tb.getDataOutput();
ftr.reset(accessor, tIndex);
for (int i = 0; i < aggs.length; i++) {
- try {
- int begin = tb.getSize();
- if (aggs[i] == null) {
- aggs[i] = aggFactories[i].createAggregateEvaluator(ctx);
- }
- aggs[i].init(output);
- tb.addFieldEndOffset();
- stateFieldLength[i] = tb.getSize() - begin;
- } catch (AlgebricksException e) {
- throw new HyracksDataException(e);
+ int begin = tb.getSize();
+ if (aggs[i] == null) {
+ aggs[i] = aggFactories[i].createAggregateEvaluator(ctx);
}
+ aggs[i].init(output);
+ tb.addFieldEndOffset();
+ stateFieldLength[i] = tb.getSize() - begin;
}
// doing initial aggregate
ftr.reset(accessor, tIndex);
for (int i = 0; i < aggs.length; i++) {
- try {
- byte[] data = tb.getByteArray();
- int prevFieldPos = i + keys.length - 1;
- int start = prevFieldPos >= 0 ? tb.getFieldEndOffsets()[prevFieldPos] : 0;
- aggs[i].step(ftr, data, start, stateFieldLength[i]);
- } catch (AlgebricksException e) {
- throw new HyracksDataException(e);
- }
+ byte[] data = tb.getByteArray();
+ int prevFieldPos = i + keys.length - 1;
+ int start = prevFieldPos >= 0 ? tb.getFieldEndOffsets()[prevFieldPos] : 0;
+ aggs[i].step(ftr, data, start, stateFieldLength[i]);
}
}
@@ -101,14 +92,10 @@
int stateTupleStart = stateAccessor.getTupleStartOffset(stateTupleIndex);
int fieldSlotLength = stateAccessor.getFieldSlotsLength();
for (int i = 0; i < aggs.length; i++) {
- try {
- byte[] data = stateAccessor.getBuffer().array();
- int start = stateAccessor.getFieldStartOffset(stateTupleIndex, i + keys.length)
- + stateTupleStart + fieldSlotLength;
- aggs[i].step(ftr, data, start, stateFieldLength[i]);
- } catch (AlgebricksException e) {
- throw new HyracksDataException(e);
- }
+ byte[] data = stateAccessor.getBuffer().array();
+ int start = stateAccessor.getFieldStartOffset(stateTupleIndex, i + keys.length)
+ + stateTupleStart + fieldSlotLength;
+ aggs[i].step(ftr, data, start, stateFieldLength[i]);
}
}
@@ -121,13 +108,9 @@
int refOffset = startOffset + stateAccessor.getFieldSlotsLength() + aggFieldOffset;
int start = refOffset;
for (int i = 0; i < aggs.length; i++) {
- try {
- aggs[i].finishPartial(data, start, stateFieldLength[i], tb.getDataOutput());
- start += stateFieldLength[i];
- tb.addFieldEndOffset();
- } catch (AlgebricksException e) {
- throw new HyracksDataException(e);
- }
+ aggs[i].finishPartial(data, start, stateFieldLength[i], tb.getDataOutput());
+ start += stateFieldLength[i];
+ tb.addFieldEndOffset();
}
return true;
}
@@ -141,13 +124,9 @@
int refOffset = startOffset + stateAccessor.getFieldSlotsLength() + aggFieldOffset;
int start = refOffset;
for (int i = 0; i < aggs.length; i++) {
- try {
- aggs[i].finish(data, start, stateFieldLength[i], tb.getDataOutput());
- start += stateFieldLength[i];
- tb.addFieldEndOffset();
- } catch (AlgebricksException e) {
- throw new HyracksDataException(e);
- }
+ aggs[i].finish(data, start, stateFieldLength[i], tb.getDataOutput());
+ start += stateFieldLength[i];
+ tb.addFieldEndOffset();
}
return true;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java
index 13e3646..87f3cb4 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java
@@ -18,7 +18,6 @@
*/
package org.apache.hyracks.algebricks.runtime.operators.aggreg;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
import org.apache.hyracks.api.comm.IFrameTupleAccessor;
@@ -58,20 +57,12 @@
// initialize aggregate functions
for (int i = 0; i < agg.length; i++) {
- try {
- agg[i].init();
- } catch (AlgebricksException e) {
- throw new HyracksDataException(e);
- }
+ agg[i].init();
}
ftr.reset(accessor, tIndex);
for (int i = 0; i < agg.length; i++) {
- try {
- agg[i].step(ftr);
- } catch (AlgebricksException e) {
- throw new HyracksDataException(e);
- }
+ agg[i].step(ftr);
}
}
@@ -81,11 +72,7 @@
IAggregateEvaluator[] agg = (IAggregateEvaluator[]) state.state;
ftr.reset(accessor, tIndex);
for (int i = 0; i < agg.length; i++) {
- try {
- agg[i].step(ftr);
- } catch (AlgebricksException e) {
- throw new HyracksDataException(e);
- }
+ agg[i].step(ftr);
}
}
@@ -94,25 +81,17 @@
int tIndex, AggregateState state) throws HyracksDataException {
IAggregateEvaluator[] agg = (IAggregateEvaluator[]) state.state;
for (int i = 0; i < agg.length; i++) {
- try {
- agg[i].finish(p);
- tupleBuilder.addField(p.getByteArray(), p.getStartOffset(), p.getLength());
- } catch (AlgebricksException e) {
- throw new HyracksDataException(e);
- }
+ agg[i].finish(p);
+ tupleBuilder.addField(p.getByteArray(), p.getStartOffset(), p.getLength());
}
return true;
}
@Override
- public AggregateState createAggregateStates() {
+ public AggregateState createAggregateStates() throws HyracksDataException {
IAggregateEvaluator[] agg = new IAggregateEvaluator[aggFactories.length];
for (int i = 0; i < agg.length; i++) {
- try {
- agg[i] = aggFactories[i].createAggregateEvaluator(ctx);
- } catch (AlgebricksException e) {
- throw new IllegalStateException(e);
- }
+ agg[i] = aggFactories[i].createAggregateEvaluator(ctx);
}
return new AggregateState(agg);
}
@@ -127,12 +106,8 @@
int tIndex, AggregateState state) throws HyracksDataException {
IAggregateEvaluator[] agg = (IAggregateEvaluator[]) state.state;
for (int i = 0; i < agg.length; i++) {
- try {
- agg[i].finishPartial(p);
- tupleBuilder.addField(p.getByteArray(), p.getStartOffset(), p.getLength());
- } catch (AlgebricksException e) {
- throw new HyracksDataException(e);
- }
+ agg[i].finishPartial(p);
+ tupleBuilder.addField(p.getByteArray(), p.getStartOffset(), p.getLength());
}
return true;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
index 9fea189..32eff3a 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
@@ -18,7 +18,6 @@
*/
package org.apache.hyracks.algebricks.runtime.operators.base;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -35,11 +34,11 @@
}
@Override
- public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws AlgebricksException, HyracksDataException {
+ public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException {
return createOneOutputPushRuntime(ctx);
}
public abstract AbstractOneInputOneOutputPushRuntime createOneOutputPushRuntime(IHyracksTaskContext ctx)
- throws AlgebricksException, HyracksDataException;
+ throws HyracksDataException;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/SinkRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/SinkRuntimeFactory.java
index d1fb703..a838557 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/SinkRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/SinkRuntimeFactory.java
@@ -20,7 +20,6 @@
import java.nio.ByteBuffer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -39,7 +38,7 @@
}
@Override
- public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException {
return new AbstractOneInputSinkPushRuntime() {
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java
index 2368a9c..4678887 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java
@@ -20,7 +20,6 @@
import java.nio.ByteBuffer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputPushRuntime;
import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
@@ -58,7 +57,7 @@
@Override
public AbstractOneInputOneOutputPushRuntime createOneOutputPushRuntime(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
for (int i = 0; i < comparatorFactories.length; ++i) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/AlgebricksMetaOperatorDescriptor.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/AlgebricksMetaOperatorDescriptor.java
index 10c8d66..0b7cb7e 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/AlgebricksMetaOperatorDescriptor.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/AlgebricksMetaOperatorDescriptor.java
@@ -20,7 +20,6 @@
import java.nio.ByteBuffer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
import org.apache.hyracks.api.comm.IFrameWriter;
@@ -100,17 +99,11 @@
PipelineAssembler pa = new PipelineAssembler(pipeline, inputArity, outputArity, null,
pipelineOutputRecordDescriptor);
- try {
- startOfPipeline = pa.assemblePipeline(writer, ctx);
- } catch (AlgebricksException e) {
- throw new HyracksDataException(e);
- }
+ startOfPipeline = pa.assemblePipeline(writer, ctx);
try {
startOfPipeline.open();
- } catch (HyracksDataException e) {
- // Tell the downstream the job fails.
+ } catch (Exception e) {
startOfPipeline.fail();
- // Throws the exception.
throw e;
} finally {
startOfPipeline.close();
@@ -134,11 +127,7 @@
.getInputRecordDescriptor(AlgebricksMetaOperatorDescriptor.this.getActivityId(), 0);
PipelineAssembler pa = new PipelineAssembler(pipeline, inputArity, outputArity,
pipelineInputRecordDescriptor, pipelineOutputRecordDescriptor);
- try {
- startOfPipeline = pa.assemblePipeline(writer, ctx);
- } catch (AlgebricksException ae) {
- throw new HyracksDataException(ae);
- }
+ startOfPipeline = pa.assemblePipeline(writer, ctx);
}
startOfPipeline.open();
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java
index cd83c94..03e2aaf 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java
@@ -18,7 +18,6 @@
*/
package org.apache.hyracks.algebricks.runtime.operators.meta;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
import org.apache.hyracks.api.comm.IFrameWriter;
@@ -45,7 +44,7 @@
this.outputArity = outputArity;
}
- public IFrameWriter assemblePipeline(IFrameWriter writer, IHyracksTaskContext ctx) throws AlgebricksException,
+ public IFrameWriter assemblePipeline(IFrameWriter writer, IHyracksTaskContext ctx) throws
HyracksDataException {
// plug the operators
IFrameWriter start = writer;// this.writer;
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
index 9a9fb72..8b13e09 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
@@ -21,7 +21,6 @@
import java.io.DataOutput;
import java.nio.ByteBuffer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
@@ -70,7 +69,7 @@
@Override
public AbstractOneInputOneOutputPushRuntime createOneOutputPushRuntime(final IHyracksTaskContext ctx)
- throws AlgebricksException, HyracksDataException {
+ throws HyracksDataException {
RecordDescriptor pipelineOutputRecordDescriptor = null;
@@ -160,8 +159,14 @@
for (int t = 0; t < nTuple; t++) {
tRef.reset(tAccess, t);
startOfPipeline.writeTuple(buffer, t);
- startOfPipeline.open();
- startOfPipeline.close();
+ try {
+ startOfPipeline.open();
+ } catch (Exception e) {
+ startOfPipeline.fail();
+ throw e;
+ } finally {
+ startOfPipeline.close();
+ }
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
index df05d50..3e30f73 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
@@ -20,7 +20,6 @@
import java.nio.ByteBuffer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputPushRuntime;
import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
@@ -57,10 +56,8 @@
@Override
public AbstractOneInputOneOutputPushRuntime createOneOutputPushRuntime(final IHyracksTaskContext ctx)
- throws AlgebricksException {
-
+ throws HyracksDataException {
return new AbstractOneInputOneOutputPushRuntime() {
-
FrameSorterMergeSort frameSorter = null;
@Override
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 88a2f24..520465a 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
@@ -22,7 +22,6 @@
import java.nio.ByteBuffer;
import java.util.Arrays;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
@@ -87,7 +86,7 @@
@Override
public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
final int[] projectionToOutColumns = new int[projectionList.length];
for (int j = 0; j < projectionList.length; j++) {
projectionToOutColumns[j] = Arrays.binarySearch(outColumns, projectionList[j]);
@@ -108,11 +107,7 @@
first = false;
int n = evalFactories.length;
for (int i = 0; i < n; i++) {
- try {
- eval[i] = evalFactories[i].createScalarEvaluator(ctx);
- } catch (AlgebricksException ae) {
- throw new HyracksDataException(ae);
- }
+ eval[i] = evalFactories[i].createScalarEvaluator(ctx);
}
}
isOpen = true;
@@ -177,7 +172,7 @@
tb.addField(accessor, tIndex, projectionList[f]);
}
}
- } catch (HyracksDataException | AlgebricksException e) {
+ } catch (HyracksDataException e) {
throw new HyracksDataException(ErrorCode.HYRACKS, ErrorCode.ERROR_PROCESSING_TUPLE,
"Error evaluating tuple %1$s in AssignRuntime", (Throwable) e,
new Serializable[] { tupleIndex });
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java
index bb6cc73..38fe7d1 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java
@@ -21,7 +21,6 @@
import java.nio.ByteBuffer;
import java.util.Arrays;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
@@ -79,7 +78,7 @@
@Override
public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
final int[] projectionToOutColumns = new int[projectionList.length];
for (int j = 0; j < projectionList.length; j++) {
projectionToOutColumns[j] = Arrays.binarySearch(outColumns, projectionList[j]);
@@ -99,19 +98,11 @@
first = false;
int n = runningAggregates.length;
for (int i = 0; i < n; i++) {
- try {
- raggs[i] = runningAggregates[i].createRunningAggregateEvaluator(ctx);
- } catch (AlgebricksException ae) {
- throw new HyracksDataException(ae);
- }
+ raggs[i] = runningAggregates[i].createRunningAggregateEvaluator(ctx);
}
}
for (int i = 0; i < runningAggregates.length; i++) {
- try {
- raggs[i].init();
- } catch (AlgebricksException ae) {
- throw new HyracksDataException(ae);
- }
+ raggs[i].init();
}
isOpen = true;
writer.open();
@@ -148,11 +139,7 @@
for (int f = 0; f < projectionList.length; f++) {
int k = projectionToOutColumns[f];
if (k >= 0) {
- try {
- raggs[k].step(tupleRef, p);
- } catch (AlgebricksException e) {
- throw new HyracksDataException(e);
- }
+ raggs[k].step(tupleRef, p);
tb.addField(p.getByteArray(), p.getStartOffset(), p.getLength());
} else {
tb.addField(accessor, tIndex, projectionList[f]);
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SinkWriterRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SinkWriterRuntimeFactory.java
index 62e3542..d52ceee 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SinkWriterRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SinkWriterRuntimeFactory.java
@@ -20,11 +20,10 @@
import java.io.BufferedOutputStream;
import java.io.File;
-import java.io.FileNotFoundException;
import java.io.FileOutputStream;
+import java.io.IOException;
import java.io.PrintStream;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.data.IAWriter;
import org.apache.hyracks.algebricks.data.IAWriterFactory;
import org.apache.hyracks.algebricks.data.IPrinterFactory;
@@ -32,6 +31,7 @@
import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SinkWriterRuntimeFactory implements IPushRuntimeFactory {
@@ -67,14 +67,13 @@
}
@Override
- public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws AlgebricksException {
- PrintStream filePrintStream = null;
+ public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException {
try {
- filePrintStream = new PrintStream(new BufferedOutputStream(new FileOutputStream(outputFile)));
- } catch (FileNotFoundException e) {
- throw new AlgebricksException(e);
+ PrintStream filePrintStream = new PrintStream(new BufferedOutputStream(new FileOutputStream(outputFile)));
+ IAWriter w = writerFactory.createWriter(fields, filePrintStream, printerFactories, inputRecordDesc);
+ return new SinkWriterRuntime(w, filePrintStream, inputRecordDesc, true);
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
}
- IAWriter w = writerFactory.createWriter(fields, filePrintStream, printerFactories, inputRecordDesc);
- return new SinkWriterRuntime(w, filePrintStream, inputRecordDesc, true);
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SplitOperatorDescriptor.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SplitOperatorDescriptor.java
index 2215a96..be39208 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SplitOperatorDescriptor.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SplitOperatorDescriptor.java
@@ -20,7 +20,6 @@
import java.nio.ByteBuffer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.data.IBinaryIntegerInspector;
import org.apache.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
@@ -93,11 +92,7 @@
final FrameTupleReference tRef = new FrameTupleReference();;
final IBinaryIntegerInspector intInsepctor = intInsepctorFactory.createBinaryIntegerInspector(ctx);
final IScalarEvaluator eval;
- try {
- eval = brachingExprEvalFactory.createScalarEvaluator(ctx);
- } catch (AlgebricksException ae) {
- throw new HyracksDataException(ae);
- }
+ eval = brachingExprEvalFactory.createScalarEvaluator(ctx);
for (int i = 0; i < numberOfNonMaterializedOutputs; i++) {
appenders[i] = new FrameTupleAppender(new VSizeFrame(ctx), true);
}
@@ -122,11 +117,7 @@
for (int i = 0; i < tupleCount; i++) {
// Get the output branch number from the field in the given tuple.
tRef.reset(accessor, i);
- try {
- eval.evaluate(tRef, p);
- } catch (AlgebricksException ae) {
- throw new HyracksDataException(ae);
- }
+ eval.evaluate(tRef, p);
outputBranch = intInsepctor.getIntegerValue(p.getByteArray(), p.getStartOffset(),
p.getLength());
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamLimitRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamLimitRuntimeFactory.java
index 11f47ac..59df402 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamLimitRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamLimitRuntimeFactory.java
@@ -20,7 +20,6 @@
import java.nio.ByteBuffer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.data.IBinaryIntegerInspector;
import org.apache.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
@@ -76,13 +75,9 @@
writer.open();
if (evalMaxObjects == null) {
initAccessAppendRef(ctx);
- try {
- evalMaxObjects = maxObjectsEvalFactory.createScalarEvaluator(ctx);
- if (offsetEvalFactory != null) {
- evalOffset = offsetEvalFactory.createScalarEvaluator(ctx);
- }
- } catch (AlgebricksException ae) {
- throw new HyracksDataException(ae);
+ evalMaxObjects = maxObjectsEvalFactory.createScalarEvaluator(ctx);
+ if (offsetEvalFactory != null) {
+ evalOffset = offsetEvalFactory.createScalarEvaluator(ctx);
}
}
afterLastTuple = false;
@@ -138,11 +133,7 @@
private int evaluateInteger(IScalarEvaluator eval, int tIdx) throws HyracksDataException {
tRef.reset(tAccess, tIdx);
- try {
- eval.evaluate(tRef, p);
- } catch (AlgebricksException ae) {
- throw new HyracksDataException(ae);
- }
+ eval.evaluate(tRef, p);
int lim = bii.getIntegerValue(p.getByteArray(), p.getStartOffset(), p.getLength());
return lim;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamProjectRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamProjectRuntimeFactory.java
index 43c63b5..a8ca082 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamProjectRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamProjectRuntimeFactory.java
@@ -21,7 +21,6 @@
import java.nio.ByteBuffer;
import java.util.Arrays;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -48,10 +47,8 @@
@Override
public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final IHyracksTaskContext ctx)
- throws AlgebricksException {
-
+ throws HyracksDataException {
return new AbstractOneInputOneOutputOneFramePushRuntime() {
-
private boolean first = true;
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java
index 418ea23..0f57fd7 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java
@@ -21,7 +21,6 @@
import java.io.DataOutput;
import java.nio.ByteBuffer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.data.IBinaryBooleanInspector;
import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
@@ -92,11 +91,7 @@
public void open() throws HyracksDataException {
if (eval == null) {
initAccessAppendFieldRef(ctx);
- try {
- eval = cond.createScalarEvaluator(ctx);
- } catch (AlgebricksException ae) {
- throw new HyracksDataException(ae);
- }
+ eval = cond.createScalarEvaluator(ctx);
}
isOpen = true;
writer.open();
@@ -135,11 +130,7 @@
int nTuple = tAccess.getTupleCount();
for (int t = 0; t < nTuple; t++) {
tRef.reset(tAccess, t);
- try {
- eval.evaluate(tRef, p);
- } catch (AlgebricksException ae) {
- throw new HyracksDataException(ae);
- }
+ eval.evaluate(tRef, p);
if (bbi.getBooleanValue(p.getByteArray(), p.getStartOffset(), p.getLength())) {
if (projectionList != null) {
appendProjectionToFrame(t, projectionList);
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StringStreamingRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StringStreamingRuntimeFactory.java
index 36d3349..024f6f5 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StringStreamingRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StringStreamingRuntimeFactory.java
@@ -25,7 +25,6 @@
import java.io.PrintStream;
import java.nio.ByteBuffer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.data.IPrinter;
import org.apache.hyracks.algebricks.data.IPrinterFactory;
import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
@@ -55,14 +54,13 @@
@Override
public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
final IPrinter[] printers = new IPrinter[printerFactories.length];
for (int i = 0; i < printerFactories.length; i++) {
printers[i] = printerFactories[i].createPrinter();
}
return new AbstractOneInputOneOutputOneFramePushRuntime() {
-
final class ForwardScriptOutput implements Runnable {
private InputStream inStream;
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
index 300be34..2410d67 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
@@ -24,7 +24,6 @@
import java.io.IOException;
import java.nio.ByteBuffer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IUnnestingPositionWriter;
@@ -79,16 +78,12 @@
@Override
public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final IHyracksTaskContext ctx)
- throws AlgebricksException {
+ throws HyracksDataException {
ByteArrayOutputStream bos = new ByteArrayOutputStream();
DataOutput output = new DataOutputStream(bos);
if (missingWriterFactory != null) {
IMissingWriter missingWriter = missingWriterFactory.createMissingWriter();
- try {
- missingWriter.writeMissing(output);
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
+ missingWriter.writeMissing(output);
}
byte[] missingBytes = bos.toByteArray();
int missingBytesLen = bos.size();
@@ -114,13 +109,13 @@
try {
unnest.init(tRef);
unnesting(t);
- } catch (AlgebricksException | IOException ae) {
+ } catch (IOException ae) {
throw new HyracksDataException(ae);
}
}
}
- private void unnesting(int t) throws AlgebricksException, IOException {
+ private void unnesting(int t) throws IOException {
// Assumes that when unnesting the tuple, each step() call for each element
// in the tuple will increase the positionIndex, and the positionIndex will
// be reset when a new tuple is to be processed.
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/pom.xml b/hyracks-fullstack/algebricks/algebricks-tests/pom.xml
index 54bf66a..42b946d 100644
--- a/hyracks-fullstack/algebricks/algebricks-tests/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-tests/pom.xml
@@ -125,11 +125,6 @@
</dependency>
<dependency>
<groupId>org.apache.hyracks</groupId>
- <artifactId>algebricks-common</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-dataflow-common</artifactId>
<version>${project.version}</version>
</dependency>
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntArrayUnnester.java b/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntArrayUnnester.java
index 7cce35e..a789ad0 100644
--- a/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntArrayUnnester.java
+++ b/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntArrayUnnester.java
@@ -20,10 +20,10 @@
import java.io.IOException;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -39,19 +39,19 @@
private static final long serialVersionUID = 1L;
@Override
- public IUnnestingEvaluator createUnnestingEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IUnnestingEvaluator createUnnestingEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
final ArrayBackedValueStorage abvs = new ArrayBackedValueStorage();
return new IUnnestingEvaluator() {
private int pos;
@Override
- public void init(IFrameTupleReference tuple) throws AlgebricksException {
+ public void init(IFrameTupleReference tuple) throws HyracksDataException {
pos = 0;
}
@Override
- public boolean step(IPointable result) throws AlgebricksException {
+ public boolean step(IPointable result) throws HyracksDataException {
try {
if (pos < x.length) {
// Writes one byte to distinguish between null
@@ -66,7 +66,7 @@
}
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntegerAddEvalFactory.java b/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntegerAddEvalFactory.java
index f7a97f4..ea332df 100644
--- a/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntegerAddEvalFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntegerAddEvalFactory.java
@@ -24,6 +24,7 @@
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.IntegerPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -43,7 +44,7 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private IPointable p = VoidPointable.FACTORY.createPointable();
private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
@@ -52,7 +53,7 @@
private IScalarEvaluator evalRight = evalRightFactory.createScalarEvaluator(ctx);
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
evalLeft.evaluate(tuple, p);
int v1 = IntegerPointable.getInteger(p.getByteArray(), p.getStartOffset());
evalRight.evaluate(tuple, p);
@@ -62,7 +63,7 @@
argOut.getDataOutput().writeInt(v1 + v2);
result.set(argOut);
} catch (IOException e) {
- throw new AlgebricksException(e);
+ throw new HyracksDataException(e);
}
}
};
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntegerConstantEvalFactory.java b/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntegerConstantEvalFactory.java
index fe5bbf6..26790c5 100644
--- a/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntegerConstantEvalFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntegerConstantEvalFactory.java
@@ -18,7 +18,6 @@
*/
package org.apache.hyracks.algebricks.tests.pushruntime;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -43,21 +42,17 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private ArrayBackedValueStorage buf = new ArrayBackedValueStorage();
{
- try {
IntegerSerializerDeserializer.INSTANCE.serialize(value, buf.getDataOutput());
- } catch (HyracksDataException e) {
- throw new AlgebricksException(e);
- }
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
result.set(buf);
}
};
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntegerEqualsEvalFactory.java b/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntegerEqualsEvalFactory.java
index 106f611..3cdd1f0 100644
--- a/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntegerEqualsEvalFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntegerEqualsEvalFactory.java
@@ -18,10 +18,10 @@
*/
package org.apache.hyracks.algebricks.tests.pushruntime;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.BooleanPointable;
import org.apache.hyracks.data.std.primitive.IntegerPointable;
@@ -40,7 +40,7 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private IPointable p = VoidPointable.FACTORY.createPointable();
private IScalarEvaluator eval1 = evalFact1.createScalarEvaluator(ctx);
@@ -48,7 +48,7 @@
private byte[] rBytes = new byte[1];
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval1.evaluate(tuple, p);
int v1 = IntegerPointable.getInteger(p.getByteArray(), p.getStartOffset());
eval2.evaluate(tuple, p);
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntegerGreaterThanEvalFactory.java b/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntegerGreaterThanEvalFactory.java
index 400dd23..2ed4dd3 100644
--- a/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntegerGreaterThanEvalFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-tests/src/main/java/org/apache/hyracks/algebricks/tests/pushruntime/IntegerGreaterThanEvalFactory.java
@@ -18,10 +18,10 @@
*/
package org.apache.hyracks.algebricks.tests.pushruntime;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.BooleanPointable;
import org.apache.hyracks.data.std.primitive.IntegerPointable;
@@ -40,7 +40,7 @@
}
@Override
- public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
private IPointable p = VoidPointable.FACTORY.createPointable();
private IScalarEvaluator eval1 = evalFact1.createScalarEvaluator(ctx);
@@ -48,7 +48,7 @@
private byte[] rBytes = new byte[1];
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
eval1.evaluate(tuple, p);
int v1 = IntegerPointable.getInteger(p.getByteArray(), p.getStartOffset());
eval2.evaluate(tuple, p);
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePairComparatorFactory.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePairComparatorFactory.java
index 1437087..9800f5b 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePairComparatorFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePairComparatorFactory.java
@@ -21,7 +21,8 @@
import java.io.Serializable;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public interface ITuplePairComparatorFactory extends Serializable {
- public ITuplePairComparator createTuplePairComparator(IHyracksTaskContext ctx);
+ public ITuplePairComparator createTuplePairComparator(IHyracksTaskContext ctx) throws HyracksDataException;
}
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 d17af4c..a79ab1d 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
@@ -20,12 +20,15 @@
import java.io.Serializable;
import java.util.Formatter;
+import java.util.logging.Level;
+import java.util.logging.Logger;
/**
* The main execution time exception type for runtime errors in a hyracks environment
*/
public class HyracksDataException extends HyracksException {
private static final long serialVersionUID = 1L;
+ private static final Logger LOGGER = Logger.getLogger(HyracksDataException.class.getName());
public static final String NONE = "";
public static final int UNKNOWN = 0;
@@ -45,10 +48,6 @@
this.params = params;
}
- public HyracksDataException() {
- this(NONE, UNKNOWN, null, null, null, new Serializable[0]);
- }
-
public HyracksDataException(String message) {
this(NONE, UNKNOWN, message, (Throwable) null, (String) null);
}
@@ -136,6 +135,10 @@
}
fmt.format(message == null ? "null" : message, (Object[]) params);
return fmt.out().toString();
+ } catch (Exception e) {
+ // Do not throw further exceptions during exception processing.
+ LOGGER.log(Level.WARNING, e.getLocalizedMessage(), e);
+ return e.getMessage();
}
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/IAggregatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/IAggregatorDescriptor.java
index 9552294..32e2f7b 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/IAggregatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/IAggregatorDescriptor.java
@@ -27,13 +27,14 @@
/**
* Create an aggregate state
*
- * @return
+ * @return an aggregate state
+ * @throws HyracksDataException
*/
- AggregateState createAggregateStates();
+ AggregateState createAggregateStates() throws HyracksDataException;
/**
* Initialize the state based on the input tuple.
- *
+ *
* @param tupleBuilder
* @param accessor
* @param tIndex
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
index 4fa1498..09b7544 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
@@ -112,7 +112,8 @@
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+ IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+ throws HyracksDataException {
final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(nljAid, 0);
final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
final ITuplePairComparator comparator = comparatorFactory.createTuplePairComparator(ctx);