[NO ISSUE][FAIL] Replace MetadataException by AlgebricksException in APIs
Change-Id: Ia543ac84b698042582249837912d21ddaa48453f
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2134
Sonar-Qube: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Michael Blow <mblow@apache.org>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index cd0a63c..12114f0 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -100,19 +100,15 @@
jobGenParams.readFromFuncArgs(unnestFuncExpr.getArguments());
MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
- Dataset dataset;
- try {
- dataset = metadataProvider.findDataset(jobGenParams.getDataverseName(), jobGenParams.getDatasetName());
- } catch (MetadataException e) {
- throw new AlgebricksException(e);
- }
+ Dataset dataset = metadataProvider.findDataset(jobGenParams.getDataverseName(), jobGenParams.getDatasetName());
int[] keyIndexes = getKeyIndexes(jobGenParams.getKeyVarList(), inputSchemas);
int[] minFilterFieldIndexes = getKeyIndexes(unnestMapOp.getMinFilterVars(), inputSchemas);
int[] maxFilterFieldIndexes = getKeyIndexes(unnestMapOp.getMaxFilterVars(), inputSchemas);
boolean retainNull = false;
if (op.getOperatorTag() == LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP) {
- // By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching tuples.
+ // By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching
+ // tuples.
retainNull = true;
}
// Build runtime.
@@ -135,46 +131,38 @@
AbstractUnnestMapOperator unnestMap, IOperatorSchema opSchema, boolean retainInput, boolean retainMissing,
String datasetName, Dataset dataset, String indexName, ATypeTag searchKeyType, int[] keyFields,
SearchModifierType searchModifierType, IAlgebricksConstantValue similarityThreshold,
- int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
- boolean isFullTextSearchQuery) throws AlgebricksException {
- try {
-
- boolean propagateIndexFilter = unnestMap.propagateIndexFilter();
- IAObject simThresh = ((AsterixConstantValue) similarityThreshold).getObject();
- int numPrimaryKeys = dataset.getPrimaryKeys().size();
- Index secondaryIndex = MetadataManager.INSTANCE
- .getIndex(metadataProvider.getMetadataTxnContext(), dataset.getDataverseName(),
- dataset.getDatasetName(), indexName);
- if (secondaryIndex == null) {
- throw new AlgebricksException(
- "Code generation error: no index " + indexName + " for dataset " + datasetName);
- }
- IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(unnestMap);
- RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint =
- metadataProvider.getSplitProviderAndConstraints(dataset, indexName);
- // TODO: Here we assume there is only one search key field.
- int queryField = keyFields[0];
- // Get tokenizer and search modifier factories.
- IInvertedIndexSearchModifierFactory searchModifierFactory =
- InvertedIndexAccessMethod.getSearchModifierFactory(searchModifierType, simThresh, secondaryIndex);
- IBinaryTokenizerFactory queryTokenizerFactory = InvertedIndexAccessMethod
- .getBinaryTokenizerFactory(searchModifierType, searchKeyType, secondaryIndex);
- IIndexDataflowHelperFactory dataflowHelperFactory =
- new IndexDataflowHelperFactory(metadataProvider.getStorageComponentProvider().getStorageManager(),
- secondarySplitsAndConstraint.first);
- LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp =
- new LSMInvertedIndexSearchOperatorDescriptor(jobSpec, outputRecDesc, queryField,
- dataflowHelperFactory, queryTokenizerFactory, searchModifierFactory, retainInput,
- retainMissing, context.getMissingWriterFactory(),
- dataset.getSearchCallbackFactory(metadataProvider.getStorageComponentProvider(),
- secondaryIndex,
- ((JobEventListenerFactory) jobSpec.getJobletEventListenerFactory()).getJobId(),
- IndexOperation.SEARCH, null), minFilterFieldIndexes, maxFilterFieldIndexes,
- isFullTextSearchQuery, numPrimaryKeys, propagateIndexFilter);
- return new Pair<>(invIndexSearchOp, secondarySplitsAndConstraint.second);
- } catch (MetadataException e) {
- throw new AlgebricksException(e);
+ int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean isFullTextSearchQuery)
+ throws AlgebricksException {
+ boolean propagateIndexFilter = unnestMap.propagateIndexFilter();
+ IAObject simThresh = ((AsterixConstantValue) similarityThreshold).getObject();
+ int numPrimaryKeys = dataset.getPrimaryKeys().size();
+ Index secondaryIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
+ dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+ if (secondaryIndex == null) {
+ throw new AlgebricksException(
+ "Code generation error: no index " + indexName + " for dataset " + datasetName);
}
+ IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(unnestMap);
+ RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint =
+ metadataProvider.getSplitProviderAndConstraints(dataset, indexName);
+ // TODO: Here we assume there is only one search key field.
+ int queryField = keyFields[0];
+ // Get tokenizer and search modifier factories.
+ IInvertedIndexSearchModifierFactory searchModifierFactory =
+ InvertedIndexAccessMethod.getSearchModifierFactory(searchModifierType, simThresh, secondaryIndex);
+ IBinaryTokenizerFactory queryTokenizerFactory =
+ InvertedIndexAccessMethod.getBinaryTokenizerFactory(searchModifierType, searchKeyType, secondaryIndex);
+ IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
+ metadataProvider.getStorageComponentProvider().getStorageManager(), secondarySplitsAndConstraint.first);
+ LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
+ jobSpec, outputRecDesc, queryField, dataflowHelperFactory, queryTokenizerFactory, searchModifierFactory,
+ retainInput, retainMissing, context.getMissingWriterFactory(),
+ dataset.getSearchCallbackFactory(metadataProvider.getStorageComponentProvider(), secondaryIndex,
+ ((JobEventListenerFactory) jobSpec.getJobletEventListenerFactory()).getJobId(),
+ IndexOperation.SEARCH, null),
+ minFilterFieldIndexes, maxFilterFieldIndexes, isFullTextSearchQuery, numPrimaryKeys,
+ propagateIndexFilter);
+ return new Pair<>(invIndexSearchOp, secondarySplitsAndConstraint.second);
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index 6f3ec76..4579e6f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -245,7 +245,8 @@
AssignOperator assign = new AssignOperator(pkVars, pkExprs);
assign.getInputs().add(new MutableObject<>(dssOp));
- // If the input is pre-sorted, we set the ordering property explicitly in the assign
+ // If the input is pre-sorted, we set the ordering property explicitly in the
+ // assign
if (clffs.alreadySorted()) {
List<OrderColumn> orderColumns = new ArrayList<>();
for (int i = 0; i < pkVars.size(); ++i) {
@@ -328,13 +329,13 @@
}
} else {
/**
- * add the collection-to-sequence right before the project,
- * because dataset only accept non-collection records
+ * add the collection-to-sequence right before the project, because dataset only
+ * accept non-collection records
*/
LogicalVariable seqVar = context.newVar();
/**
- * This assign adds a marker function collection-to-sequence: if the input is a singleton collection, unnest
- * it; otherwise do nothing.
+ * This assign adds a marker function collection-to-sequence: if the input is a
+ * singleton collection, unnest it; otherwise do nothing.
*/
AssignOperator assignCollectionToSequence = new AssignOperator(seqVar,
new MutableObject<>(new ScalarFunctionCallExpression(
@@ -557,7 +558,8 @@
return processReturningExpression(rootOperator, insertOp, compiledInsert);
}
- // Stitches the translated operators for the returning expression into the query plan.
+ // Stitches the translated operators for the returning expression into the query
+ // plan.
private ILogicalOperator processReturningExpression(ILogicalOperator inputOperator,
InsertDeleteUpsertOperator insertOp, CompiledInsertStatement compiledInsert) throws AlgebricksException {
Expression returnExpression = compiledInsert.getReturnExpression();
@@ -566,7 +568,7 @@
}
ILogicalOperator rootOperator = inputOperator;
- //Makes the id of the insert var point to the record variable.
+ // Makes the id of the insert var point to the record variable.
context.newVarFromExpression(compiledInsert.getVar());
context.setVar(compiledInsert.getVar(),
((VariableReferenceExpression) insertOp.getPayloadExpression().getValue()).getVariableReference());
@@ -606,7 +608,7 @@
dataset.getDatasetDetails(), domain);
}
- private FileSplit getDefaultOutputFileLocation(ICcApplicationContext appCtx) throws MetadataException {
+ private FileSplit getDefaultOutputFileLocation(ICcApplicationContext appCtx) throws AlgebricksException {
String outputDir = System.getProperty("java.io.tmpDir");
String filePath =
outputDir + System.getProperty("file.separator") + OUTPUT_FILE_PREFIX + outputFileID.incrementAndGet();
@@ -702,8 +704,12 @@
}
AbstractFunctionCallExpression f;
- if ((f = lookupUserDefinedFunction(signature, args)) == null) {
- f = lookupBuiltinFunction(signature.getName(), signature.getArity(), args);
+ try {
+ if ((f = lookupUserDefinedFunction(signature, args)) == null) {
+ f = lookupBuiltinFunction(signature.getName(), signature.getArity(), args);
+ }
+ } catch (AlgebricksException e) {
+ throw new CompilationException(e);
}
if (f == null) {
@@ -726,7 +732,7 @@
}
private AbstractFunctionCallExpression lookupUserDefinedFunction(FunctionSignature signature,
- List<Mutable<ILogicalExpression>> args) throws MetadataException {
+ List<Mutable<ILogicalExpression>> args) throws AlgebricksException {
if (signature.getNamespace() == null) {
return null;
}
@@ -1406,8 +1412,8 @@
}
/**
- * Eliminate shared operator references in a query plan.
- * Deep copy a new query plan subtree whenever there is a shared operator reference.
+ * Eliminate shared operator references in a query plan. Deep copy a new query
+ * plan subtree whenever there is a shared operator reference.
*
* @param plan,
* the query plan.
@@ -1421,15 +1427,16 @@
}
/**
- * Eliminate shared operator references in a query plan rooted at <code>currentOpRef.getValue()</code>.
- * Deep copy a new query plan subtree whenever there is a shared operator reference.
+ * Eliminate shared operator references in a query plan rooted at
+ * <code>currentOpRef.getValue()</code>. Deep copy a new query plan subtree
+ * whenever there is a shared operator reference.
*
* @param currentOpRef,
* the operator reference to consider
* @param opRefSet,
* the set storing seen operator references so far.
- * @return a mapping that maps old variables to new variables, for the ancestors of
- * <code>currentOpRef</code> to replace variables properly.
+ * @return a mapping that maps old variables to new variables, for the ancestors
+ * of <code>currentOpRef</code> to replace variables properly.
* @throws CompilationException
*/
private LinkedHashMap<LogicalVariable, LogicalVariable> eliminateSharedOperatorReference(
@@ -1441,9 +1448,12 @@
// Recursively eliminates shared references in nested plans.
if (currentOperator.hasNestedPlans()) {
- // Since a nested plan tree itself can never be shared with another nested plan tree in
- // another operator, the operation called in the if block does not need to replace
- // any variables further for <code>currentOpRef.getValue()</code> nor its ancestor.
+ // Since a nested plan tree itself can never be shared with another nested plan
+ // tree in
+ // another operator, the operation called in the if block does not need to
+ // replace
+ // any variables further for <code>currentOpRef.getValue()</code> nor its
+ // ancestor.
AbstractOperatorWithNestedPlans opWithNestedPlan = (AbstractOperatorWithNestedPlans) currentOperator;
for (ILogicalPlan plan : opWithNestedPlan.getNestedPlans()) {
for (Mutable<ILogicalOperator> rootRef : plan.getRoots()) {
@@ -1465,7 +1475,8 @@
LinkedHashMap<LogicalVariable, LogicalVariable> cloneVarMap =
visitor.getInputToOutputVariableMapping();
- // Substitute variables according to the deep copy which generates new variables.
+ // Substitute variables according to the deep copy which generates new
+ // variables.
VariableUtilities.substituteVariables(currentOperator, cloneVarMap, null);
varMap.putAll(cloneVarMap);
@@ -1481,7 +1492,8 @@
// Substitute variables according to the new subtree.
VariableUtilities.substituteVariables(currentOperator, childVarMap, null);
- // Updates mapping like <$a, $b> in varMap to <$a, $c>, where there is a mapping <$b, $c>
+ // Updates mapping like <$a, $b> in varMap to <$a, $c>, where there is a mapping
+ // <$b, $c>
// in childVarMap.
varMap.entrySet().forEach(entry -> {
LogicalVariable newVar = childVarMap.get(entry.getValue());
@@ -1512,7 +1524,8 @@
* the expression to select tuples that are processed by this branch.
* @param branchExpression,
* the expression to be evaluated in this branch.
- * @return a pair of the constructed subplan operator and the output variable for the branch.
+ * @return a pair of the constructed subplan operator and the output variable
+ * for the branch.
* @throws CompilationException
*/
protected Pair<ILogicalOperator, LogicalVariable> constructSubplanOperatorForBranch(ILogicalOperator inputOp,
@@ -1523,7 +1536,8 @@
Mutable<ILogicalOperator> nestedSource =
new MutableObject<>(new NestedTupleSourceOperator(new MutableObject<>(subplanOp)));
SelectOperator select = new SelectOperator(selectExpr, false, null);
- // The select operator cannot be moved up and down, otherwise it will cause typing issues (ASTERIXDB-1203).
+ // The select operator cannot be moved up and down, otherwise it will cause
+ // typing issues (ASTERIXDB-1203).
OperatorPropertiesUtil.markMovable(select, false);
select.getInputs().add(nestedSource);
Pair<ILogicalOperator, LogicalVariable> pBranch = branchExpression.accept(this, new MutableObject<>(select));
@@ -1552,12 +1566,14 @@
return new AssignOperator(v1, new MutableObject<>(comparison));
}
- // Generates the filter condition for whether a conditional branch should be executed.
+ // Generates the filter condition for whether a conditional branch should be
+ // executed.
protected Mutable<ILogicalExpression> generateNoMatchedPrecedingWhenBranchesFilter(
List<ILogicalExpression> inputBooleanExprs) {
List<Mutable<ILogicalExpression>> arguments = new ArrayList<>();
for (ILogicalExpression inputBooleanExpr : inputBooleanExprs) {
- // A NULL/MISSING valued WHEN expression does not lead to the corresponding THEN execution.
+ // A NULL/MISSING valued WHEN expression does not lead to the corresponding THEN
+ // execution.
// Therefore, we should check a previous WHEN boolean condition is not unknown.
arguments.add(generateAndNotIsUnknownWrap(inputBooleanExpr));
}
@@ -1580,7 +1596,8 @@
new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), arguments));
}
- // Generates the plan for "UNION ALL" or union expression from its input expressions.
+ // Generates the plan for "UNION ALL" or union expression from its input
+ // expressions.
protected Pair<ILogicalOperator, LogicalVariable> translateUnionAllFromInputExprs(List<ILangExpression> inputExprs,
Mutable<ILogicalOperator> tupSource) throws CompilationException {
List<Mutable<ILogicalOperator>> inputOpRefsToUnion = new ArrayList<>();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 494eb65..a811454 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -265,9 +265,11 @@
FileSplit outputFile = null;
IAWriterFactory writerFactory = PrinterBasedWriterFactory.INSTANCE;
IResultSerializerFactoryProvider resultSerializerFactoryProvider = ResultSerializerFactoryProvider.INSTANCE;
- /* Since the system runs a large number of threads, when HTTP requests don't return, it becomes difficult to
- * find the thread running the request to determine where it has stopped.
- * Setting the thread name helps make that easier
+ /*
+ * Since the system runs a large number of threads, when HTTP requests don't
+ * return, it becomes difficult to find the thread running the request to
+ * determine where it has stopped. Setting the thread name helps make that
+ * easier
*/
String threadName = Thread.currentThread().getName();
Thread.currentThread().setName(QueryTranslator.class.getSimpleName());
@@ -397,8 +399,8 @@
// No op
break;
case Statement.Kind.EXTENSION:
- ((IExtensionStatement) stmt)
- .handle(hcc, this, requestParameters, metadataProvider, resultSetIdCounter);
+ ((IExtensionStatement) stmt).handle(hcc, this, requestParameters, metadataProvider,
+ resultSetIdCounter);
break;
default:
throw new CompilationException("Unknown function");
@@ -641,7 +643,8 @@
metadataProvider.setMetadataTxnContext(mdTxnCtx);
}
- // #. add a new dataset with PendingNoOp after deleting the dataset with PendingAddOp
+ // #. add a new dataset with PendingNoOp after deleting the dataset with
+ // PendingAddOp
MetadataManager.INSTANCE.dropDataset(metadataProvider.getMetadataTxnContext(), dataverseName, datasetName);
dataset.setPendingOp(MetadataUtil.PENDING_NO_OP);
MetadataManager.INSTANCE.addDataset(metadataProvider.getMetadataTxnContext(), dataset);
@@ -656,7 +659,8 @@
// #. execute compensation operations
// remove the index in NC
// [Notice]
- // As long as we updated(and committed) metadata, we should remove any effect of the job
+ // As long as we updated(and committed) metadata, we should remove any effect of
+ // the job
// because an exception occurs during runJob.
mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
bActiveTxn = true;
@@ -790,9 +794,11 @@
int keyIndex = 0;
boolean overridesFieldTypes = false;
- // this set is used to detect duplicates in the specified keys in the create index statement
+ // this set is used to detect duplicates in the specified keys in the create
+ // index statement
// e.g. CREATE INDEX someIdx on dataset(id,id).
- // checking only the names is not enough. Need also to check the source indicators for cases like:
+ // checking only the names is not enough. Need also to check the source
+ // indicators for cases like:
// CREATE INDEX someIdx on dataset(meta().id, id)
Set<Pair<List<String>, Integer>> indexKeysSet = new HashSet<>();
@@ -820,10 +826,11 @@
throw new AsterixException(ErrorCode.INDEX_ILLEGAL_ENFORCED_NON_OPTIONAL,
String.valueOf(fieldExpr.first));
}
- // don't allow creating an enforced index on a closed-type field, fields that are part of schema.
+ // don't allow creating an enforced index on a closed-type field, fields that
+ // are part of schema.
// get the field type, if it's not null, then the field is closed-type
- if (stmtCreateIndex.isEnforced() &&
- subType.getSubFieldType(fieldExpr.first.subList(i, fieldExpr.first.size())) != null) {
+ if (stmtCreateIndex.isEnforced()
+ && subType.getSubFieldType(fieldExpr.first.subList(i, fieldExpr.first.size())) != null) {
throw new AsterixException(ErrorCode.INDEX_ILLEGAL_ENFORCED_ON_CLOSED_FIELD,
String.valueOf(fieldExpr.first));
}
@@ -845,9 +852,10 @@
"Unknown type " + (fieldExpr.second == null ? fieldExpr.first : fieldExpr.second));
}
- // try to add the key & its source to the set of keys, if key couldn't be added, there is a duplicate
- if (!indexKeysSet.add(new Pair<>(fieldExpr.first,
- stmtCreateIndex.getFieldSourceIndicators().get(keyIndex)))) {
+ // try to add the key & its source to the set of keys, if key couldn't be added,
+ // there is a duplicate
+ if (!indexKeysSet
+ .add(new Pair<>(fieldExpr.first, stmtCreateIndex.getFieldSourceIndicators().get(keyIndex)))) {
throw new AsterixException(ErrorCode.INDEX_ILLEGAL_REPETITIVE_FIELD,
String.valueOf(fieldExpr.first));
}
@@ -859,9 +867,11 @@
validateIndexKeyFields(stmtCreateIndex, keySourceIndicators, aRecordType, metaRecordType, indexFields,
indexFieldTypes);
- // Checks whether a user is trying to create an inverted secondary index on a dataset
+ // Checks whether a user is trying to create an inverted secondary index on a
+ // dataset
// with a variable-length primary key.
- // Currently, we do not support this. Therefore, as a temporary solution, we print an
+ // Currently, we do not support this. Therefore, as a temporary solution, we
+ // print an
// error message and stop.
if (stmtCreateIndex.getIndexType() == IndexType.SINGLE_PARTITION_WORD_INVIX
|| stmtCreateIndex.getIndexType() == IndexType.SINGLE_PARTITION_NGRAM_INVIX
@@ -994,8 +1004,10 @@
runJob(hcc, spec, jobFlags);
// #. flush the internal dataset
- // We need this to guarantee the correctness of component Id acceleration for secondary-to-primary index.
- // Otherwise, the new secondary index component would corresponding to a partial memory component
+ // We need this to guarantee the correctness of component Id acceleration for
+ // secondary-to-primary index.
+ // Otherwise, the new secondary index component would corresponding to a partial
+ // memory component
// of the primary index, which is incorrect.
if (ds.getDatasetType() == DatasetType.INTERNAL) {
FlushDatasetUtil.flushDataset(hcc, metadataProvider, index.getDataverseName(), index.getDatasetName());
@@ -1017,7 +1029,8 @@
bActiveTxn = true;
metadataProvider.setMetadataTxnContext(mdTxnCtx);
- // #. add another new index with PendingNoOp after deleting the index with PendingAddOp
+ // #. add another new index with PendingNoOp after deleting the index with
+ // PendingAddOp
MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), index.getDataverseName(),
index.getDatasetName(), index.getIndexName());
index.setPendingOp(MetadataUtil.PENDING_NO_OP);
@@ -1038,7 +1051,8 @@
if (bActiveTxn) {
abort(e, e, mdTxnCtx);
}
- // If files index was replicated for external dataset, it should be cleaned up on NC side
+ // If files index was replicated for external dataset, it should be cleaned up
+ // on NC side
if (filesIndexReplicated) {
mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
bActiveTxn = true;
@@ -1687,7 +1701,7 @@
}
protected boolean checkWhetherFunctionIsBeingUsed(MetadataTransactionContext ctx, String dataverseName,
- String functionName, int arity, String currentDataverse) throws MetadataException {
+ String functionName, int arity, String currentDataverse) throws AlgebricksException {
List<Dataverse> allDataverses = MetadataManager.INSTANCE.getDataverses(ctx);
for (Dataverse dataverse : allDataverses) {
if (currentDataverse != null && dataverse.getDataverseName().equals(currentDataverse)) {
@@ -1876,7 +1890,8 @@
MetadataProvider metadataProvider, InsertStatement insertUpsert)
throws RemoteException, AlgebricksException, ACIDException {
- // Insert/upsert statement rewriting (happens under the same ongoing metadata transaction)
+ // Insert/upsert statement rewriting (happens under the same ongoing metadata
+ // transaction)
Pair<IReturningStatement, Integer> rewrittenResult =
apiFramework.reWriteQuery(declaredFunctions, metadataProvider, insertUpsert, sessionOutput);
@@ -1898,7 +1913,8 @@
default:
throw new AlgebricksException("Unsupported statement type " + rewrittenInsertUpsert.getKind());
}
- // Insert/upsert statement compilation (happens under the same ongoing metadata transaction)
+ // Insert/upsert statement compilation (happens under the same ongoing metadata
+ // transaction)
return apiFramework.compileQuery(clusterInfoCollector, metadataProvider, rewrittenInsertUpsert.getQuery(),
rewrittenResult.second, datasetName, sessionOutput, clfrqs);
}
@@ -2627,7 +2643,8 @@
// all index updates has completed successfully, record transaction state
spec = ExternalIndexingOperations.buildCommitJob(ds, indexes, metadataProvider);
- // Aquire write latch again -> start a transaction and record the decision to commit
+ // Aquire write latch again -> start a transaction and record the decision to
+ // commit
mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
bActiveTxn = true;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
index 6ac9436..1ca9316 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
@@ -33,6 +33,7 @@
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.Function;
import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
@@ -65,14 +66,16 @@
}
/**
- * Retrieve stored functions (from CREATE FUNCTION statements) that have been used in an expression.
+ * Retrieve stored functions (from CREATE FUNCTION statements) that have been
+ * used in an expression.
*
* @param metadataProvider,
* the metadata provider
* @param expression,
* the expression for analysis
* @param declaredFunctions,
- * a set of declared functions in the query, which can potentially override stored functions.
+ * a set of declared functions in the query, which can potentially
+ * override stored functions.
* @param functionCollector,
* for collecting function calls in the <code>expression</code>
* @param functionParser,
@@ -85,8 +88,8 @@
Expression expression, List<FunctionSignature> declaredFunctions, List<FunctionDecl> inputFunctionDecls,
IFunctionCollector functionCollector, IFunctionParser functionParser,
IFunctionNormalizer functionNormalizer) throws CompilationException {
- List<FunctionDecl> functionDecls = inputFunctionDecls == null ? new ArrayList<>()
- : new ArrayList<>(inputFunctionDecls);
+ List<FunctionDecl> functionDecls =
+ inputFunctionDecls == null ? new ArrayList<>() : new ArrayList<>(inputFunctionDecls);
if (expression == null) {
return functionDecls;
}
@@ -102,13 +105,22 @@
}
String namespace = signature.getNamespace();
// Checks the existence of the referred dataverse.
- if (!namespace.equals(FunctionConstants.ASTERIX_NS)
- && !namespace.equals(AlgebricksBuiltinFunctions.ALGEBRICKS_NS)
- && metadataProvider.findDataverse(namespace) == null) {
- throw new CompilationException("In function call \"" + namespace + "." + signature.getName()
- + "(...)\", the dataverse \"" + namespace + "\" cannot be found!");
+ try {
+ if (!namespace.equals(FunctionConstants.ASTERIX_NS)
+ && !namespace.equals(AlgebricksBuiltinFunctions.ALGEBRICKS_NS)
+ && metadataProvider.findDataverse(namespace) == null) {
+ throw new CompilationException("In function call \"" + namespace + "." + signature.getName()
+ + "(...)\", the dataverse \"" + namespace + "\" cannot be found!");
+ }
+ } catch (AlgebricksException e) {
+ throw new CompilationException(e);
}
- Function function = lookupUserDefinedFunctionDecl(metadataProvider.getMetadataTxnContext(), signature);
+ Function function;
+ try {
+ function = lookupUserDefinedFunctionDecl(metadataProvider.getMetadataTxnContext(), signature);
+ } catch (AlgebricksException e) {
+ throw new CompilationException(e);
+ }
if (function == null) {
FunctionSignature normalizedSignature = functionNormalizer == null ? signature
: functionNormalizer.normalizeBuiltinFunctionSignature(signature);
@@ -144,7 +156,7 @@
}
private static Function lookupUserDefinedFunctionDecl(MetadataTransactionContext mdTxnCtx,
- FunctionSignature signature) throws CompilationException {
+ FunctionSignature signature) throws AlgebricksException {
if (signature.getNamespace() == null) {
return null;
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
index 23e6fb0..2e872fc 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
@@ -54,6 +54,7 @@
import org.apache.asterix.metadata.entities.NodeGroup;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.transaction.management.service.transaction.JobIdFactory;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.exceptions.HyracksDataException;
/**
@@ -95,7 +96,8 @@
private final ReadWriteLock metadataLatch;
protected boolean rebindMetadataNode = false;
- // TODO(mblow): replace references of this (non-constant) field with a method, update field name accordingly
+ // TODO(mblow): replace references of this (non-constant) field with a method,
+ // update field name accordingly
public static IMetadataManager INSTANCE;
private MetadataManager(IAsterixStateProxy proxy, IMetadataNode metadataNode) {
@@ -148,7 +150,7 @@
}
@Override
- public void addDataverse(MetadataTransactionContext ctx, Dataverse dataverse) throws MetadataException {
+ public void addDataverse(MetadataTransactionContext ctx, Dataverse dataverse) throws AlgebricksException {
try {
metadataNode.addDataverse(ctx.getJobId(), dataverse);
} catch (RemoteException e) {
@@ -158,7 +160,7 @@
}
@Override
- public void dropDataverse(MetadataTransactionContext ctx, String dataverseName) throws MetadataException {
+ public void dropDataverse(MetadataTransactionContext ctx, String dataverseName) throws AlgebricksException {
try {
metadataNode.dropDataverse(ctx.getJobId(), dataverseName);
} catch (RemoteException e) {
@@ -168,7 +170,7 @@
}
@Override
- public List<Dataverse> getDataverses(MetadataTransactionContext ctx) throws MetadataException {
+ public List<Dataverse> getDataverses(MetadataTransactionContext ctx) throws AlgebricksException {
try {
return metadataNode.getDataverses(ctx.getJobId());
} catch (RemoteException e) {
@@ -177,7 +179,7 @@
}
@Override
- public Dataverse getDataverse(MetadataTransactionContext ctx, String dataverseName) throws MetadataException {
+ public Dataverse getDataverse(MetadataTransactionContext ctx, String dataverseName) throws AlgebricksException {
// First look in the context to see if this transaction created the
// requested dataverse itself (but the dataverse is still uncommitted).
Dataverse dataverse = ctx.getDataverse(dataverseName);
@@ -211,7 +213,7 @@
@Override
public List<Dataset> getDataverseDatasets(MetadataTransactionContext ctx, String dataverseName)
- throws MetadataException {
+ throws AlgebricksException {
List<Dataset> dataverseDatasets = new ArrayList<>();
// add uncommitted temporary datasets
for (Dataset dataset : ctx.getDataverseDatasets(dataverseName)) {
@@ -238,7 +240,7 @@
}
@Override
- public void addDataset(MetadataTransactionContext ctx, Dataset dataset) throws MetadataException {
+ public void addDataset(MetadataTransactionContext ctx, Dataset dataset) throws AlgebricksException {
// add dataset into metadataNode
if (!dataset.getDatasetDetails().isTemp()) {
try {
@@ -254,7 +256,7 @@
@Override
public void dropDataset(MetadataTransactionContext ctx, String dataverseName, String datasetName)
- throws MetadataException {
+ throws AlgebricksException {
Dataset dataset = findDataset(ctx, dataverseName, datasetName);
// If a dataset is not in the cache, then it could not be a temp dataset
if (dataset == null || !dataset.getDatasetDetails().isTemp()) {
@@ -271,7 +273,7 @@
@Override
public Dataset getDataset(MetadataTransactionContext ctx, String dataverseName, String datasetName)
- throws MetadataException {
+ throws AlgebricksException {
// First look in the context to see if this transaction created the
// requested dataset itself (but the dataset is still uncommitted).
@@ -307,7 +309,7 @@
@Override
public List<Index> getDatasetIndexes(MetadataTransactionContext ctx, String dataverseName, String datasetName)
- throws MetadataException {
+ throws AlgebricksException {
List<Index> datasetIndexes = new ArrayList<>();
Dataset dataset = findDataset(ctx, dataverseName, datasetName);
if (dataset == null) {
@@ -329,7 +331,7 @@
@Override
public void addCompactionPolicy(MetadataTransactionContext mdTxnCtx, CompactionPolicy compactionPolicy)
- throws MetadataException {
+ throws AlgebricksException {
try {
metadataNode.addCompactionPolicy(mdTxnCtx.getJobId(), compactionPolicy);
} catch (RemoteException e) {
@@ -340,7 +342,7 @@
@Override
public CompactionPolicy getCompactionPolicy(MetadataTransactionContext ctx, String dataverse, String policyName)
- throws MetadataException {
+ throws AlgebricksException {
CompactionPolicy compactionPolicy;
try {
@@ -352,7 +354,7 @@
}
@Override
- public void addDatatype(MetadataTransactionContext ctx, Datatype datatype) throws MetadataException {
+ public void addDatatype(MetadataTransactionContext ctx, Datatype datatype) throws AlgebricksException {
try {
metadataNode.addDatatype(ctx.getJobId(), datatype);
} catch (RemoteException e) {
@@ -368,7 +370,7 @@
@Override
public void dropDatatype(MetadataTransactionContext ctx, String dataverseName, String datatypeName)
- throws MetadataException {
+ throws AlgebricksException {
try {
metadataNode.dropDatatype(ctx.getJobId(), dataverseName, datatypeName);
} catch (RemoteException e) {
@@ -379,7 +381,7 @@
@Override
public Datatype getDatatype(MetadataTransactionContext ctx, String dataverseName, String datatypeName)
- throws MetadataException {
+ throws AlgebricksException {
// First look in the context to see if this transaction created the
// requested datatype itself (but the datatype is still uncommitted).
Datatype datatype = ctx.getDatatype(dataverseName, datatypeName);
@@ -397,9 +399,9 @@
datatype = cache.getDatatype(dataverseName, datatypeName);
if (datatype != null) {
// Datatype is already in the cache, don't add it again.
- //create a new Datatype object with a new ARecordType object in order to avoid
- //concurrent access to UTF8StringPointable comparator in ARecordType object.
- //see issue 510
+ // create a new Datatype object with a new ARecordType object in order to avoid
+ // concurrent access to UTF8StringPointable comparator in ARecordType object.
+ // see issue 510
ARecordType aRecType = (ARecordType) datatype.getDatatype();
return new Datatype(
datatype.getDataverseName(), datatype.getDatatypeName(), new ARecordType(aRecType.getTypeName(),
@@ -420,7 +422,7 @@
}
@Override
- public void addIndex(MetadataTransactionContext ctx, Index index) throws MetadataException {
+ public void addIndex(MetadataTransactionContext ctx, Index index) throws AlgebricksException {
String dataverseName = index.getDataverseName();
String datasetName = index.getDatasetName();
Dataset dataset = findDataset(ctx, dataverseName, datasetName);
@@ -435,7 +437,7 @@
}
@Override
- public void addAdapter(MetadataTransactionContext mdTxnCtx, DatasourceAdapter adapter) throws MetadataException {
+ public void addAdapter(MetadataTransactionContext mdTxnCtx, DatasourceAdapter adapter) throws AlgebricksException {
try {
metadataNode.addAdapter(mdTxnCtx.getJobId(), adapter);
} catch (RemoteException e) {
@@ -447,10 +449,12 @@
@Override
public void dropIndex(MetadataTransactionContext ctx, String dataverseName, String datasetName, String indexName)
- throws MetadataException {
+ throws AlgebricksException {
Dataset dataset = findDataset(ctx, dataverseName, datasetName);
- // If a dataset is not in the cache, then it could be an unloaded persistent dataset.
- // If the dataset is a temp dataset, then we do not need to call any MedataNode operations.
+ // If a dataset is not in the cache, then it could be an unloaded persistent
+ // dataset.
+ // If the dataset is a temp dataset, then we do not need to call any MedataNode
+ // operations.
if (dataset == null || !dataset.getDatasetDetails().isTemp()) {
try {
metadataNode.dropIndex(ctx.getJobId(), dataverseName, datasetName, indexName);
@@ -463,7 +467,7 @@
@Override
public Index getIndex(MetadataTransactionContext ctx, String dataverseName, String datasetName, String indexName)
- throws MetadataException {
+ throws AlgebricksException {
// First look in the context to see if this transaction created the
// requested index itself (but the index is still uncommitted).
@@ -499,7 +503,7 @@
}
@Override
- public void addNode(MetadataTransactionContext ctx, Node node) throws MetadataException {
+ public void addNode(MetadataTransactionContext ctx, Node node) throws AlgebricksException {
try {
metadataNode.addNode(ctx.getJobId(), node);
} catch (RemoteException e) {
@@ -508,7 +512,7 @@
}
@Override
- public void addNodegroup(MetadataTransactionContext ctx, NodeGroup nodeGroup) throws MetadataException {
+ public void addNodegroup(MetadataTransactionContext ctx, NodeGroup nodeGroup) throws AlgebricksException {
try {
metadataNode.addNodeGroup(ctx.getJobId(), nodeGroup);
} catch (RemoteException e) {
@@ -519,7 +523,7 @@
@Override
public void dropNodegroup(MetadataTransactionContext ctx, String nodeGroupName, boolean failSilently)
- throws MetadataException {
+ throws AlgebricksException {
boolean dropped;
try {
dropped = metadataNode.dropNodegroup(ctx.getJobId(), nodeGroupName, failSilently);
@@ -532,7 +536,7 @@
}
@Override
- public NodeGroup getNodegroup(MetadataTransactionContext ctx, String nodeGroupName) throws MetadataException {
+ public NodeGroup getNodegroup(MetadataTransactionContext ctx, String nodeGroupName) throws AlgebricksException {
// First look in the context to see if this transaction created the
// requested dataverse itself (but the dataverse is still uncommitted).
NodeGroup nodeGroup = ctx.getNodeGroup(nodeGroupName);
@@ -565,7 +569,7 @@
}
@Override
- public void addFunction(MetadataTransactionContext mdTxnCtx, Function function) throws MetadataException {
+ public void addFunction(MetadataTransactionContext mdTxnCtx, Function function) throws AlgebricksException {
try {
metadataNode.addFunction(mdTxnCtx.getJobId(), function);
} catch (RemoteException e) {
@@ -576,7 +580,7 @@
@Override
public void dropFunction(MetadataTransactionContext ctx, FunctionSignature functionSignature)
- throws MetadataException {
+ throws AlgebricksException {
try {
metadataNode.dropFunction(ctx.getJobId(), functionSignature);
} catch (RemoteException e) {
@@ -587,7 +591,7 @@
@Override
public Function getFunction(MetadataTransactionContext ctx, FunctionSignature functionSignature)
- throws MetadataException {
+ throws AlgebricksException {
// First look in the context to see if this transaction created the
// requested dataset itself (but the dataset is still uncommitted).
Function function = ctx.getFunction(functionSignature);
@@ -626,9 +630,10 @@
}
@Override
- public List<Function> getFunctions(MetadataTransactionContext ctx, String dataverseName) throws MetadataException {
+ public List<Function> getFunctions(MetadataTransactionContext ctx, String dataverseName)
+ throws AlgebricksException {
try {
- return metadataNode.getFunctions(ctx.getJobId(), dataverseName);
+ return metadataNode.getFunctions(ctx.getJobId(), dataverseName);
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -636,7 +641,7 @@
@Override
public void addFeedPolicy(MetadataTransactionContext mdTxnCtx, FeedPolicyEntity feedPolicy)
- throws MetadataException {
+ throws AlgebricksException {
try {
metadataNode.addFeedPolicy(mdTxnCtx.getJobId(), feedPolicy);
} catch (RemoteException e) {
@@ -646,7 +651,7 @@
}
@Override
- public void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws MetadataException {
+ public void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws AlgebricksException {
try {
metadataNode.initializeDatasetIdFactory(ctx.getJobId());
} catch (RemoteException e) {
@@ -655,7 +660,7 @@
}
@Override
- public int getMostRecentDatasetId() throws MetadataException {
+ public int getMostRecentDatasetId() throws AlgebricksException {
try {
return metadataNode.getMostRecentDatasetId();
} catch (RemoteException e) {
@@ -665,7 +670,7 @@
@Override
public List<Function> getDataverseFunctions(MetadataTransactionContext ctx, String dataverseName)
- throws MetadataException {
+ throws AlgebricksException {
List<Function> dataverseFunctions;
try {
// Assuming that the transaction can read its own writes on the
@@ -681,7 +686,7 @@
@Override
public void dropAdapter(MetadataTransactionContext ctx, String dataverseName, String name)
- throws MetadataException {
+ throws AlgebricksException {
try {
metadataNode.dropAdapter(ctx.getJobId(), dataverseName, name);
} catch (RemoteException e) {
@@ -691,7 +696,7 @@
@Override
public DatasourceAdapter getAdapter(MetadataTransactionContext ctx, String dataverseName, String name)
- throws MetadataException {
+ throws AlgebricksException {
DatasourceAdapter adapter;
try {
adapter = metadataNode.getAdapter(ctx.getJobId(), dataverseName, name);
@@ -703,7 +708,7 @@
@Override
public void dropLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName)
- throws MetadataException {
+ throws AlgebricksException {
try {
metadataNode.dropLibrary(ctx.getJobId(), dataverseName, libraryName);
} catch (RemoteException e) {
@@ -714,7 +719,7 @@
@Override
public List<Library> getDataverseLibraries(MetadataTransactionContext ctx, String dataverseName)
- throws MetadataException {
+ throws AlgebricksException {
List<Library> dataverseLibaries;
try {
// Assuming that the transaction can read its own writes on the
@@ -729,7 +734,7 @@
}
@Override
- public void addLibrary(MetadataTransactionContext ctx, Library library) throws MetadataException {
+ public void addLibrary(MetadataTransactionContext ctx, Library library) throws AlgebricksException {
try {
metadataNode.addLibrary(ctx.getJobId(), library);
} catch (RemoteException e) {
@@ -740,7 +745,7 @@
@Override
public Library getLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
Library library;
try {
library = metadataNode.getLibrary(ctx.getJobId(), dataverseName, libraryName);
@@ -772,7 +777,7 @@
@Override
public FeedPolicyEntity getFeedPolicy(MetadataTransactionContext ctx, String dataverse, String policyName)
- throws MetadataException {
+ throws AlgebricksException {
FeedPolicyEntity feedPolicy;
try {
@@ -784,7 +789,7 @@
}
@Override
- public Feed getFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException {
+ public Feed getFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws AlgebricksException {
Feed feed;
try {
feed = metadataNode.getFeed(ctx.getJobId(), dataverse, feedName);
@@ -795,7 +800,7 @@
}
@Override
- public List<Feed> getFeeds(MetadataTransactionContext ctx, String dataverse) throws MetadataException {
+ public List<Feed> getFeeds(MetadataTransactionContext ctx, String dataverse) throws AlgebricksException {
List<Feed> feeds;
try {
feeds = metadataNode.getFeeds(ctx.getJobId(), dataverse);
@@ -806,7 +811,7 @@
}
@Override
- public void dropFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException {
+ public void dropFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws AlgebricksException {
Feed feed = null;
List<FeedConnection> feedConnections = null;
try {
@@ -824,7 +829,7 @@
}
@Override
- public void addFeed(MetadataTransactionContext ctx, Feed feed) throws MetadataException {
+ public void addFeed(MetadataTransactionContext ctx, Feed feed) throws AlgebricksException {
try {
metadataNode.addFeed(ctx.getJobId(), feed);
} catch (RemoteException e) {
@@ -835,7 +840,7 @@
@Override
public void addFeedConnection(MetadataTransactionContext ctx, FeedConnection feedConnection)
- throws MetadataException {
+ throws AlgebricksException {
try {
metadataNode.addFeedConnection(ctx.getJobId(), feedConnection);
} catch (RemoteException e) {
@@ -846,7 +851,7 @@
@Override
public void dropFeedConnection(MetadataTransactionContext ctx, String dataverseName, String feedName,
- String datasetName) throws MetadataException {
+ String datasetName) throws AlgebricksException {
try {
metadataNode.dropFeedConnection(ctx.getJobId(), dataverseName, feedName, datasetName);
} catch (RemoteException e) {
@@ -857,7 +862,7 @@
@Override
public FeedConnection getFeedConnection(MetadataTransactionContext ctx, String dataverseName, String feedName,
- String datasetName) throws MetadataException {
+ String datasetName) throws AlgebricksException {
try {
return metadataNode.getFeedConnection(ctx.getJobId(), dataverseName, feedName, datasetName);
} catch (RemoteException e) {
@@ -867,7 +872,7 @@
@Override
public List<FeedConnection> getFeedConections(MetadataTransactionContext ctx, String dataverseName, String feedName)
- throws MetadataException {
+ throws AlgebricksException {
try {
return metadataNode.getFeedConnections(ctx.getJobId(), dataverseName, feedName);
} catch (RemoteException e) {
@@ -877,7 +882,7 @@
@Override
public List<DatasourceAdapter> getDataverseAdapters(MetadataTransactionContext mdTxnCtx, String dataverse)
- throws MetadataException {
+ throws AlgebricksException {
List<DatasourceAdapter> dataverseAdapters;
try {
dataverseAdapters = metadataNode.getDataverseAdapters(mdTxnCtx.getJobId(), dataverse);
@@ -889,7 +894,7 @@
@Override
public void dropFeedPolicy(MetadataTransactionContext mdTxnCtx, String dataverseName, String policyName)
- throws MetadataException {
+ throws AlgebricksException {
FeedPolicyEntity feedPolicy;
try {
feedPolicy = metadataNode.getFeedPolicy(mdTxnCtx.getJobId(), dataverseName, policyName);
@@ -901,7 +906,7 @@
}
public List<FeedPolicyEntity> getDataversePolicies(MetadataTransactionContext mdTxnCtx, String dataverse)
- throws MetadataException {
+ throws AlgebricksException {
List<FeedPolicyEntity> dataverseFeedPolicies;
try {
dataverseFeedPolicies = metadataNode.getDataversePolicies(mdTxnCtx.getJobId(), dataverse);
@@ -913,7 +918,7 @@
@Override
public List<ExternalFile> getDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset)
- throws MetadataException {
+ throws AlgebricksException {
List<ExternalFile> externalFiles;
try {
externalFiles = metadataNode.getExternalFiles(mdTxnCtx.getJobId(), dataset);
@@ -924,7 +929,7 @@
}
@Override
- public void addExternalFile(MetadataTransactionContext ctx, ExternalFile externalFile) throws MetadataException {
+ public void addExternalFile(MetadataTransactionContext ctx, ExternalFile externalFile) throws AlgebricksException {
try {
metadataNode.addExternalFile(ctx.getJobId(), externalFile);
} catch (RemoteException e) {
@@ -933,7 +938,7 @@
}
@Override
- public void dropExternalFile(MetadataTransactionContext ctx, ExternalFile externalFile) throws MetadataException {
+ public void dropExternalFile(MetadataTransactionContext ctx, ExternalFile externalFile) throws AlgebricksException {
try {
metadataNode.dropExternalFile(ctx.getJobId(), externalFile.getDataverseName(),
externalFile.getDatasetName(), externalFile.getFileNumber());
@@ -944,7 +949,7 @@
@Override
public ExternalFile getExternalFile(MetadataTransactionContext ctx, String dataverseName, String datasetName,
- Integer fileNumber) throws MetadataException {
+ Integer fileNumber) throws AlgebricksException {
ExternalFile file;
try {
file = metadataNode.getExternalFile(ctx.getJobId(), dataverseName, datasetName, fileNumber);
@@ -954,10 +959,10 @@
return file;
}
- //TODO: Optimize <-- use keys instead of object -->
+ // TODO: Optimize <-- use keys instead of object -->
@Override
public void dropDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset)
- throws MetadataException {
+ throws AlgebricksException {
try {
metadataNode.dropExternalFiles(mdTxnCtx.getJobId(), dataset);
} catch (RemoteException e) {
@@ -966,7 +971,7 @@
}
@Override
- public void updateDataset(MetadataTransactionContext ctx, Dataset dataset) throws MetadataException {
+ public void updateDataset(MetadataTransactionContext ctx, Dataset dataset) throws AlgebricksException {
try {
metadataNode.updateDataset(ctx.getJobId(), dataset);
} catch (RemoteException e) {
@@ -992,7 +997,7 @@
@Override
public <T extends IExtensionMetadataEntity> void addEntity(MetadataTransactionContext mdTxnCtx, T entity)
- throws MetadataException {
+ throws AlgebricksException {
try {
metadataNode.addEntity(mdTxnCtx.getJobId(), entity);
} catch (RemoteException e) {
@@ -1002,7 +1007,7 @@
@Override
public <T extends IExtensionMetadataEntity> void upsertEntity(MetadataTransactionContext mdTxnCtx, T entity)
- throws MetadataException {
+ throws AlgebricksException {
try {
metadataNode.upsertEntity(mdTxnCtx.getJobId(), entity);
} catch (RemoteException e) {
@@ -1012,7 +1017,7 @@
@Override
public <T extends IExtensionMetadataEntity> void deleteEntity(MetadataTransactionContext mdTxnCtx, T entity)
- throws MetadataException {
+ throws AlgebricksException {
try {
metadataNode.deleteEntity(mdTxnCtx.getJobId(), entity);
} catch (RemoteException e) {
@@ -1022,7 +1027,7 @@
@Override
public <T extends IExtensionMetadataEntity> List<T> getEntities(MetadataTransactionContext mdTxnCtx,
- IExtensionMetadataSearchKey searchKey) throws MetadataException {
+ IExtensionMetadataSearchKey searchKey) throws AlgebricksException {
try {
return metadataNode.getEntities(mdTxnCtx.getJobId(), searchKey);
} catch (RemoteException e) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index c183416..3b084c9 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -34,7 +34,6 @@
import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.common.dataflow.LSMIndexUtil;
import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
import org.apache.asterix.common.transactions.AbstractOperationCallback;
@@ -101,6 +100,7 @@
import org.apache.asterix.transaction.management.opcallbacks.UpsertOperationCallback;
import org.apache.asterix.transaction.management.service.transaction.DatasetIdFactory;
import org.apache.asterix.transaction.management.service.transaction.TransactionContext;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -207,15 +207,15 @@
* @param entity
* @param tupleTranslator
* @param index
- * @throws MetadataException
+ * @throws AlgebricksException
*/
private <T> void addEntity(JobId jobId, T entity, IMetadataEntityTupleTranslator<T> tupleTranslator,
- IMetadataIndex index) throws MetadataException {
+ IMetadataIndex index) throws AlgebricksException {
try {
ITupleReference tuple = tupleTranslator.getTupleFromMetadataEntity(entity);
insertTupleIntoIndex(jobId, index, tuple);
} catch (HyracksDataException | ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@@ -226,15 +226,15 @@
* @param entity
* @param tupleTranslator
* @param index
- * @throws MetadataException
+ * @throws AlgebricksException
*/
private <T> void upsertEntity(JobId jobId, T entity, IMetadataEntityTupleTranslator<T> tupleTranslator,
- IMetadataIndex index) throws MetadataException {
+ IMetadataIndex index) throws AlgebricksException {
try {
ITupleReference tuple = tupleTranslator.getTupleFromMetadataEntity(entity);
upsertTupleIntoIndex(jobId, index, tuple);
} catch (HyracksDataException | ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@@ -245,15 +245,15 @@
* @param entity
* @param tupleTranslator
* @param index
- * @throws MetadataException
+ * @throws AlgebricksException
*/
private <T> void deleteEntity(JobId jobId, T entity, IMetadataEntityTupleTranslator<T> tupleTranslator,
- IMetadataIndex index) throws MetadataException {
+ IMetadataIndex index) throws AlgebricksException {
try {
ITupleReference tuple = tupleTranslator.getTupleFromMetadataEntity(entity);
deleteTupleFromIndex(jobId, index, tuple);
} catch (HyracksDataException | ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@@ -265,29 +265,29 @@
* @param tupleTranslator
* @param index
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
private <T> List<T> getEntities(JobId jobId, ITupleReference searchKey,
IMetadataEntityTupleTranslator<T> tupleTranslator, IMetadataIndex index)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
IValueExtractor<T> valueExtractor = new MetadataEntityValueExtractor<>(tupleTranslator);
List<T> results = new ArrayList<>();
searchIndex(jobId, index, searchKey, valueExtractor, results);
return results;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@SuppressWarnings("unchecked")
@Override
public <T extends IExtensionMetadataEntity> void addEntity(JobId jobId, T entity)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
ExtensionMetadataDataset<T> index = (ExtensionMetadataDataset<T>) extensionDatasets.get(entity.getDatasetId());
if (index == null) {
- throw new MetadataException("Metadata Extension Index: " + entity.getDatasetId() + " was not found");
+ throw new AlgebricksException("Metadata Extension Index: " + entity.getDatasetId() + " was not found");
}
IMetadataEntityTupleTranslator<T> tupleTranslator = index.getTupleTranslator();
addEntity(jobId, entity, tupleTranslator, index);
@@ -296,10 +296,10 @@
@SuppressWarnings("unchecked")
@Override
public <T extends IExtensionMetadataEntity> void upsertEntity(JobId jobId, T entity)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
ExtensionMetadataDataset<T> index = (ExtensionMetadataDataset<T>) extensionDatasets.get(entity.getDatasetId());
if (index == null) {
- throw new MetadataException("Metadata Extension Index: " + entity.getDatasetId() + " was not found");
+ throw new AlgebricksException("Metadata Extension Index: " + entity.getDatasetId() + " was not found");
}
IMetadataEntityTupleTranslator<T> tupleTranslator = index.getTupleTranslator();
upsertEntity(jobId, entity, tupleTranslator, index);
@@ -308,10 +308,10 @@
@SuppressWarnings("unchecked")
@Override
public <T extends IExtensionMetadataEntity> void deleteEntity(JobId jobId, T entity)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
ExtensionMetadataDataset<T> index = (ExtensionMetadataDataset<T>) extensionDatasets.get(entity.getDatasetId());
if (index == null) {
- throw new MetadataException("Metadata Extension Index: " + entity.getDatasetId() + " was not found");
+ throw new AlgebricksException("Metadata Extension Index: " + entity.getDatasetId() + " was not found");
}
IMetadataEntityTupleTranslator<T> tupleTranslator = index.getTupleTranslator();
deleteEntity(jobId, entity, tupleTranslator, index);
@@ -320,36 +320,36 @@
@SuppressWarnings("unchecked")
@Override
public <T extends IExtensionMetadataEntity> List<T> getEntities(JobId jobId, IExtensionMetadataSearchKey searchKey)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
ExtensionMetadataDataset<T> index =
(ExtensionMetadataDataset<T>) extensionDatasets.get(searchKey.getDatasetId());
if (index == null) {
- throw new MetadataException("Metadata Extension Index: " + searchKey.getDatasetId() + " was not found");
+ throw new AlgebricksException("Metadata Extension Index: " + searchKey.getDatasetId() + " was not found");
}
IMetadataEntityTupleTranslator<T> tupleTranslator = index.getTupleTranslator();
return getEntities(jobId, searchKey.getSearchKey(), tupleTranslator, index);
}
@Override
- public void addDataverse(JobId jobId, Dataverse dataverse) throws MetadataException, RemoteException {
+ public void addDataverse(JobId jobId, Dataverse dataverse) throws AlgebricksException, RemoteException {
try {
DataverseTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDataverseTupleTranslator(true);
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(dataverse);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
- throw new MetadataException(
+ throw new AlgebricksException(
"A dataverse with this name " + dataverse.getDataverseName() + " already exists.", e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public void addDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException {
+ public void addDataset(JobId jobId, Dataset dataset) throws AlgebricksException, RemoteException {
try {
// Insert into the 'dataset' dataset.
DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(true);
@@ -367,70 +367,70 @@
}
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
- throw new MetadataException("A dataset with this name " + dataset.getDatasetName()
+ throw new AlgebricksException("A dataset with this name " + dataset.getDatasetName()
+ " already exists in dataverse '" + dataset.getDataverseName() + "'.", e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public void addIndex(JobId jobId, Index index) throws MetadataException, RemoteException {
+ public void addIndex(JobId jobId, Index index) throws AlgebricksException, RemoteException {
try {
IndexTupleTranslator tupleWriter = tupleTranslatorProvider.getIndexTupleTranslator(jobId, this, true);
ITupleReference tuple = tupleWriter.getTupleFromMetadataEntity(index);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
- throw new MetadataException("An index with name '" + index.getIndexName() + "' already exists.", e);
+ throw new AlgebricksException("An index with name '" + index.getIndexName() + "' already exists.", e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public void addNode(JobId jobId, Node node) throws MetadataException, RemoteException {
+ public void addNode(JobId jobId, Node node) throws AlgebricksException, RemoteException {
try {
NodeTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getNodeTupleTranslator(true);
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(node);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODE_DATASET, tuple);
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
- throw new MetadataException("A node with name '" + node.getNodeName() + "' already exists.", e);
+ throw new AlgebricksException("A node with name '" + node.getNodeName() + "' already exists.", e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws MetadataException, RemoteException {
+ public void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws AlgebricksException, RemoteException {
try {
NodeGroupTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getNodeGroupTupleTranslator(true);
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(nodeGroup);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
- throw new MetadataException(
+ throw new AlgebricksException(
"A nodegroup with name '" + nodeGroup.getNodeGroupName() + "' already exists.", e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public void addDatatype(JobId jobId, Datatype datatype) throws MetadataException, RemoteException {
+ public void addDatatype(JobId jobId, Datatype datatype) throws AlgebricksException, RemoteException {
try {
DatatypeTupleTranslator tupleReaderWriter =
tupleTranslatorProvider.getDataTypeTupleTranslator(jobId, this, true);
@@ -438,18 +438,18 @@
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
- throw new MetadataException("A datatype with name '" + datatype.getDatatypeName() + "' already exists.",
- e);
+ throw new AlgebricksException(
+ "A datatype with name '" + datatype.getDatatypeName() + "' already exists.", e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public void addFunction(JobId jobId, Function function) throws MetadataException, RemoteException {
+ public void addFunction(JobId jobId, Function function) throws AlgebricksException, RemoteException {
try {
// Insert into the 'function' dataset.
FunctionTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFunctionTupleTranslator(true);
@@ -459,14 +459,14 @@
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
- throw new MetadataException("A function with this name " + function.getName() + " and arity "
+ throw new AlgebricksException("A function with this name " + function.getName() + " and arity "
+ function.getArity() + " already exists in dataverse '" + function.getDataverseName() + "'.",
e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@@ -495,10 +495,13 @@
// TODO: fix exceptions once new BTree exception model is in hyracks.
indexAccessor.forceInsert(tuple);
- //Manually complete the operation after the insert. This is to decrement the resource counters within the
- //index that determine how many tuples are still 'in-flight' within the index. Normally the log flusher
- //does this. The only exception is the index registered as the "primary" which we will let be decremented
- //by the job commit log event
+ // Manually complete the operation after the insert. This is to decrement the
+ // resource counters within the
+ // index that determine how many tuples are still 'in-flight' within the index.
+ // Normally the log flusher
+ // does this. The only exception is the index registered as the "primary" which
+ // we will let be decremented
+ // by the job commit log event
if (!((TransactionContext) txnCtx).getPrimaryIndexOpTracker().equals(lsmIndex.getOperationTracker())) {
lsmIndex.getOperationTracker().completeOperation(lsmIndex, LSMOperationType.FORCE_MODIFICATION, null,
modCallback);
@@ -529,10 +532,13 @@
metadataIndex.isPrimaryIndex());
LSMIndexUtil.checkAndSetFirstLSN((AbstractLSMIndex) lsmIndex, transactionSubsystem.getLogManager());
indexAccessor.forceUpsert(tuple);
- //Manually complete the operation after the insert. This is to decrement the resource counters within the
- //index that determine how many tuples are still 'in-flight' within the index. Normally the log flusher
- //does this. The only exception is the index registered as the "primary" which we will let be decremented
- //by the job commit log event
+ // Manually complete the operation after the insert. This is to decrement the
+ // resource counters within the
+ // index that determine how many tuples are still 'in-flight' within the index.
+ // Normally the log flusher
+ // does this. The only exception is the index registered as the "primary" which
+ // we will let be decremented
+ // by the job commit log event
if (!((TransactionContext) txnCtx).getPrimaryIndexOpTracker().equals(lsmIndex.getOperationTracker())) {
lsmIndex.getOperationTracker().completeOperation(lsmIndex, LSMOperationType.FORCE_MODIFICATION, null,
modCallback);
@@ -546,9 +552,12 @@
IMetadataIndex metadataIndex, ILSMIndex lsmIndex, Operation indexOp) throws ACIDException {
ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
- // Regardless of the index type (primary or secondary index), secondary index modification callback is given
- // This is still correct since metadata index operation doesn't require any lock from ConcurrentLockMgr and
- // The difference between primaryIndexModCallback and secondaryIndexModCallback is that primary index requires
+ // Regardless of the index type (primary or secondary index), secondary index
+ // modification callback is given
+ // This is still correct since metadata index operation doesn't require any lock
+ // from ConcurrentLockMgr and
+ // The difference between primaryIndexModCallback and secondaryIndexModCallback
+ // is that primary index requires
// locks and secondary index doesn't.
return new SecondaryIndexModificationOperationCallback(metadataIndex.getDatasetId(),
metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
@@ -556,7 +565,7 @@
}
@Override
- public void dropDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
+ public void dropDataverse(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException {
try {
confirmDataverseCanBeDeleted(jobId, dataverseName);
@@ -570,7 +579,7 @@
dropDataset(jobId, dataverseName, ds.getDatasetName());
}
- //After dropping datasets, drop datatypes
+ // After dropping datasets, drop datatypes
List<Datatype> dataverseDatatypes;
// As a side effect, acquires an S lock on the 'datatype' dataset
// on behalf of txnId.
@@ -630,22 +639,22 @@
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS)
&& e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
- throw new MetadataException("Cannot drop dataverse '" + dataverseName + "' because it doesn't exist.",
+ throw new AlgebricksException("Cannot drop dataverse '" + dataverseName + "' because it doesn't exist.",
e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public void dropDataset(JobId jobId, String dataverseName, String datasetName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
Dataset dataset = getDataset(jobId, dataverseName, datasetName);
if (dataset == null) {
- throw new MetadataException("Cannot drop dataset '" + datasetName + "' because it doesn't exist.");
+ throw new AlgebricksException("Cannot drop dataset '" + datasetName + "' because it doesn't exist.");
}
try {
// Delete entry from the 'datasets' dataset.
@@ -681,19 +690,19 @@
// artifacts.
if (!hde.getComponent().equals(ErrorCode.HYRACKS)
|| hde.getErrorCode() != ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
- throw new MetadataException(hde);
+ throw new AlgebricksException(hde);
}
} finally {
deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
}
} catch (HyracksDataException | ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public void dropIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, datasetName, indexName);
// Searches the index for the tuple to be deleted. Acquires an S
@@ -705,19 +714,19 @@
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS)
&& e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
- throw new MetadataException(
+ throw new AlgebricksException(
"Cannot drop index '" + datasetName + "." + indexName + "' because it doesn't exist.", e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public boolean dropNodegroup(JobId jobId, String nodeGroupName, boolean failSilently)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
List<String> datasetNames = getDatasetNamesPartitionedOnThisNodeGroup(jobId, nodeGroupName);
if (!datasetNames.isEmpty()) {
if (failSilently) {
@@ -729,7 +738,7 @@
for (int i = 0; i < datasetNames.size(); i++) {
sb.append("\n" + (i + 1) + "- " + datasetNames.get(i) + ".");
}
- throw new MetadataException(sb.toString());
+ throw new AlgebricksException(sb.toString());
}
try {
ITupleReference searchKey = createTuple(nodeGroupName);
@@ -743,19 +752,19 @@
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS)
&& e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
- throw new MetadataException("Cannot drop nodegroup '" + nodeGroupName + "' because it doesn't exist",
+ throw new AlgebricksException("Cannot drop nodegroup '" + nodeGroupName + "' because it doesn't exist",
e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public void dropDatatype(JobId jobId, String dataverseName, String datatypeName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
confirmDatatypeIsUnused(jobId, dataverseName, datatypeName);
@@ -780,17 +789,17 @@
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS)
&& e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
- throw new MetadataException("Cannot drop type '" + datatypeName + "' because it doesn't exist", e);
+ throw new AlgebricksException("Cannot drop type '" + datatypeName + "' because it doesn't exist", e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
private void forceDropDatatype(JobId jobId, String dataverseName, String datatypeName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, datatypeName);
// Searches the index for the tuple to be deleted. Acquires an S
@@ -802,12 +811,12 @@
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS)
&& e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
- throw new MetadataException("Cannot drop type '" + datatypeName + "' because it doesn't exist", e);
+ throw new AlgebricksException("Cannot drop type '" + datatypeName + "' because it doesn't exist", e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@@ -833,10 +842,13 @@
LSMIndexUtil.checkAndSetFirstLSN((AbstractLSMIndex) lsmIndex, transactionSubsystem.getLogManager());
indexAccessor.forceDelete(tuple);
- //Manually complete the operation after the insert. This is to decrement the resource counters within the
- //index that determine how many tuples are still 'in-flight' within the index. Normally the log flusher
- //does this. The only exception is the index registered as the "primary" which we will let be decremented
- //by the job commit log event
+ // Manually complete the operation after the insert. This is to decrement the
+ // resource counters within the
+ // index that determine how many tuples are still 'in-flight' within the index.
+ // Normally the log flusher
+ // does this. The only exception is the index registered as the "primary" which
+ // we will let be decremented
+ // by the job commit log event
if (!((TransactionContext) txnCtx).getPrimaryIndexOpTracker().equals(lsmIndex.getOperationTracker())) {
lsmIndex.getOperationTracker().completeOperation(lsmIndex, LSMOperationType.FORCE_MODIFICATION, null,
modCallback);
@@ -847,7 +859,7 @@
}
@Override
- public List<Dataverse> getDataverses(JobId jobId) throws MetadataException, RemoteException {
+ public List<Dataverse> getDataverses(JobId jobId) throws AlgebricksException, RemoteException {
try {
DataverseTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDataverseTupleTranslator(false);
IValueExtractor<Dataverse> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
@@ -855,12 +867,12 @@
searchIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, null, valueExtractor, results);
return results;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public Dataverse getDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
+ public Dataverse getDataverse(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
DataverseTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDataverseTupleTranslator(false);
@@ -872,13 +884,13 @@
}
return results.get(0);
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(false);
@@ -887,12 +899,12 @@
searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
return results;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public List<Feed> getDataverseFeeds(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
+ public List<Feed> getDataverseFeeds(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
FeedTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedTupleTranslator(false);
@@ -901,13 +913,13 @@
searchIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, searchKey, valueExtractor, results);
return results;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public List<Library> getDataverseLibraries(JobId jobId, String dataverseName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
LibraryTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getLibraryTupleTranslator(false);
@@ -916,12 +928,12 @@
searchIndex(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, searchKey, valueExtractor, results);
return results;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
private List<Datatype> getDataverseDatatypes(JobId jobId, String dataverseName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
DatatypeTupleTranslator tupleReaderWriter =
@@ -931,13 +943,13 @@
searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
return results;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public Dataset getDataset(JobId jobId, String dataverseName, String datasetName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, datasetName);
DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(false);
@@ -949,11 +961,11 @@
}
return results.get(0);
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
- public List<Dataset> getAllDatasets(JobId jobId) throws MetadataException, RemoteException {
+ public List<Dataset> getAllDatasets(JobId jobId) throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = null;
DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(false);
@@ -962,11 +974,11 @@
searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
return results;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
- public List<Datatype> getAllDatatypes(JobId jobId) throws MetadataException, RemoteException {
+ public List<Datatype> getAllDatatypes(JobId jobId) throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = null;
DatatypeTupleTranslator tupleReaderWriter =
@@ -976,55 +988,57 @@
searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
return results;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
private void confirmDataverseCanBeDeleted(JobId jobId, String dataverseName)
- throws MetadataException, RemoteException {
- //If a dataset from a DIFFERENT dataverse
- //uses a type from this dataverse
- //throw an error
+ throws AlgebricksException, RemoteException {
+ // If a dataset from a DIFFERENT dataverse
+ // uses a type from this dataverse
+ // throw an error
List<Dataset> datasets = getAllDatasets(jobId);
for (Dataset set : datasets) {
if (set.getDataverseName().equals(dataverseName)) {
continue;
}
if (set.getItemTypeDataverseName().equals(dataverseName)) {
- throw new MetadataException("Cannot drop dataverse. Type " + dataverseName + "." + set.getItemTypeName()
- + " used by dataset " + set.getDataverseName() + "." + set.getDatasetName());
+ throw new AlgebricksException(
+ "Cannot drop dataverse. Type " + dataverseName + "." + set.getItemTypeName()
+ + " used by dataset " + set.getDataverseName() + "." + set.getDatasetName());
}
}
}
private void confirmDatatypeIsUnused(JobId jobId, String dataverseName, String datatypeName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
confirmDatatypeIsUnusedByDatatypes(jobId, dataverseName, datatypeName);
confirmDatatypeIsUnusedByDatasets(jobId, dataverseName, datatypeName);
}
private void confirmDatatypeIsUnusedByDatasets(JobId jobId, String dataverseName, String datatypeName)
- throws MetadataException, RemoteException {
- //If any dataset uses this type, throw an error
+ throws AlgebricksException, RemoteException {
+ // If any dataset uses this type, throw an error
List<Dataset> datasets = getAllDatasets(jobId);
for (Dataset set : datasets) {
if (set.getItemTypeName().equals(datatypeName) && set.getItemTypeDataverseName().equals(dataverseName)) {
- throw new MetadataException("Cannot drop type " + dataverseName + "." + datatypeName
+ throw new AlgebricksException("Cannot drop type " + dataverseName + "." + datatypeName
+ " being used by dataset " + set.getDataverseName() + "." + set.getDatasetName());
}
}
}
private void confirmDatatypeIsUnusedByDatatypes(JobId jobId, String dataverseName, String datatypeName)
- throws MetadataException, RemoteException {
- //If any datatype uses this type, throw an error
- //TODO: Currently this loads all types into memory. This will need to be fixed for large numbers of types
+ throws AlgebricksException, RemoteException {
+ // If any datatype uses this type, throw an error
+ // TODO: Currently this loads all types into memory. This will need to be fixed
+ // for large numbers of types
Datatype dataTypeToBeDropped = getDatatype(jobId, dataverseName, datatypeName);
assert dataTypeToBeDropped != null;
IAType typeToBeDropped = dataTypeToBeDropped.getDatatype();
List<Datatype> datatypes = getAllDatatypes(jobId);
for (Datatype dataType : datatypes) {
- //skip types in different dataverses as well as the type to be dropped itself
+ // skip types in different dataverses as well as the type to be dropped itself
if (!dataType.getDataverseName().equals(dataverseName)
|| dataType.getDatatype().getTypeName().equals(datatypeName)) {
continue;
@@ -1032,15 +1046,15 @@
AbstractComplexType recType = (AbstractComplexType) dataType.getDatatype();
if (recType.containsType(typeToBeDropped)) {
- throw new MetadataException("Cannot drop type " + dataverseName + "." + datatypeName
+ throw new AlgebricksException("Cannot drop type " + dataverseName + "." + datatypeName
+ " being used by type " + dataverseName + "." + recType.getTypeName());
}
}
}
private List<String> getNestedComplexDatatypeNamesForThisDatatype(JobId jobId, String dataverseName,
- String datatypeName) throws MetadataException, RemoteException {
- //Return all field types that aren't builtin types
+ String datatypeName) throws AlgebricksException, RemoteException {
+ // Return all field types that aren't builtin types
Datatype parentType = getDatatype(jobId, dataverseName, datatypeName);
List<IAType> subTypes = null;
@@ -1064,8 +1078,9 @@
}
public List<String> getDatasetNamesPartitionedOnThisNodeGroup(JobId jobId, String nodegroup)
- throws MetadataException, RemoteException {
- //this needs to scan the datasets and return the datasets that use this nodegroup
+ throws AlgebricksException, RemoteException {
+ // this needs to scan the datasets and return the datasets that use this
+ // nodegroup
List<String> nodeGroupDatasets = new ArrayList<>();
List<Dataset> datasets = getAllDatasets(jobId);
for (Dataset set : datasets) {
@@ -1079,7 +1094,7 @@
@Override
public Index getIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, datasetName, indexName);
IndexTupleTranslator tupleReaderWriter =
@@ -1092,13 +1107,13 @@
}
return results.get(0);
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public List<Index> getDatasetIndexes(JobId jobId, String dataverseName, String datasetName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, datasetName);
IndexTupleTranslator tupleReaderWriter =
@@ -1108,13 +1123,13 @@
searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
return results;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public Datatype getDatatype(JobId jobId, String dataverseName, String datatypeName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, datatypeName);
DatatypeTupleTranslator tupleReaderWriter =
@@ -1127,12 +1142,12 @@
}
return results.get(0);
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public NodeGroup getNodeGroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException {
+ public NodeGroup getNodeGroup(JobId jobId, String nodeGroupName) throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(nodeGroupName);
NodeGroupTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getNodeGroupTupleTranslator(false);
@@ -1144,13 +1159,13 @@
}
return results.get(0);
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public Function getFunction(JobId jobId, FunctionSignature functionSignature)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName(),
"" + functionSignature.getArity());
@@ -1163,12 +1178,12 @@
}
return results.get(0);
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public List<Function> getFunctions(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
+ public List<Function> getFunctions(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
FunctionTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFunctionTupleTranslator(false);
@@ -1177,18 +1192,18 @@
searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
return results;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public void dropFunction(JobId jobId, FunctionSignature functionSignature)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
Function function = getFunction(jobId, functionSignature);
if (function == null) {
- throw new MetadataException(
+ throw new AlgebricksException(
"Cannot drop function '" + functionSignature.toString() + "' because it doesn't exist.");
}
try {
@@ -1206,18 +1221,18 @@
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS)
&& e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
- throw new MetadataException("There is no function with the name " + functionSignature.getName()
+ throw new AlgebricksException("There is no function with the name " + functionSignature.getName()
+ " and arity " + functionSignature.getArity(), e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
private ITupleReference getTupleToBeDeleted(JobId jobId, IMetadataIndex metadataIndex, ITupleReference searchKey)
- throws MetadataException, HyracksDataException, RemoteException {
+ throws AlgebricksException, HyracksDataException, RemoteException {
IValueExtractor<ITupleReference> valueExtractor = new TupleCopyValueExtractor(metadataIndex.getTypeTraits());
List<ITupleReference> results = new ArrayList<>();
searchIndex(jobId, metadataIndex, searchKey, valueExtractor, results);
@@ -1307,10 +1322,10 @@
private <ResultType> void searchIndex(JobId jobId, IMetadataIndex index, ITupleReference searchKey,
IValueExtractor<ResultType> valueExtractor, List<ResultType> results)
- throws MetadataException, HyracksDataException, RemoteException {
+ throws AlgebricksException, HyracksDataException, RemoteException {
IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
if (index.getFile() == null) {
- throw new MetadataException("No file for Index " + index.getDataverseName() + "." + index.getIndexName());
+ throw new AlgebricksException("No file for Index " + index.getDataverseName() + "." + index.getIndexName());
}
String resourceName = index.getFile().getRelativePath();
IIndex indexInstance = datasetLifecycleManager.get(resourceName);
@@ -1346,7 +1361,7 @@
}
@Override
- public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException {
+ public void initializeDatasetIdFactory(JobId jobId) throws AlgebricksException, RemoteException {
int mostRecentDatasetId = MetadataIndexImmutableProperties.FIRST_AVAILABLE_USER_DATASET_ID;
try {
String resourceName = MetadataPrimaryIndexes.DATASET_DATASET.getFile().getRelativePath();
@@ -1381,7 +1396,7 @@
}
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
DatasetIdFactory.initialize(mostRecentDatasetId);
@@ -1412,7 +1427,7 @@
@Override
public List<Function> getDataverseFunctions(JobId jobId, String dataverseName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
FunctionTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFunctionTupleTranslator(false);
@@ -1421,12 +1436,12 @@
searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
return results;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public void addAdapter(JobId jobId, DatasourceAdapter adapter) throws MetadataException, RemoteException {
+ public void addAdapter(JobId jobId, DatasourceAdapter adapter) throws AlgebricksException, RemoteException {
try {
// Insert into the 'Adapter' dataset.
DatasourceAdapterTupleTranslator tupleReaderWriter =
@@ -1435,22 +1450,22 @@
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, adapterTuple);
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
- throw new MetadataException("A adapter with this name " + adapter.getAdapterIdentifier().getName()
+ throw new AlgebricksException("A adapter with this name " + adapter.getAdapterIdentifier().getName()
+ " already exists in dataverse '" + adapter.getAdapterIdentifier().getNamespace() + "'.", e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public void dropAdapter(JobId jobId, String dataverseName, String adapterName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
DatasourceAdapter adapter = getAdapter(jobId, dataverseName, adapterName);
if (adapter == null) {
- throw new MetadataException("Cannot drop adapter '" + adapter + "' because it doesn't exist.");
+ throw new AlgebricksException("Cannot drop adapter '" + adapter + "' because it doesn't exist.");
}
try {
// Delete entry from the 'Adapter' dataset.
@@ -1466,19 +1481,19 @@
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS)
&& e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
- throw new MetadataException("Cannot drop adapter '" + adapterName + " since it doesn't exist", e);
+ throw new AlgebricksException("Cannot drop adapter '" + adapterName + " since it doesn't exist", e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public DatasourceAdapter getAdapter(JobId jobId, String dataverseName, String adapterName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, adapterName);
DatasourceAdapterTupleTranslator tupleReaderWriter =
@@ -1491,13 +1506,13 @@
}
return results.get(0);
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public void addCompactionPolicy(JobId jobId, CompactionPolicy compactionPolicy)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
// Insert into the 'CompactionPolicy' dataset.
CompactionPolicyTupleTranslator tupleReaderWriter =
@@ -1506,19 +1521,19 @@
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET, compactionPolicyTuple);
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
- throw new MetadataException("A compcation policy with this name " + compactionPolicy.getPolicyName()
+ throw new AlgebricksException("A compcation policy with this name " + compactionPolicy.getPolicyName()
+ " already exists in dataverse '" + compactionPolicy.getPolicyName() + "'.", e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public CompactionPolicy getCompactionPolicy(JobId jobId, String dataverse, String policyName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverse, policyName);
CompactionPolicyTupleTranslator tupleReaderWriter =
@@ -1531,13 +1546,13 @@
}
return null;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName);
DatasourceAdapterTupleTranslator tupleReaderWriter =
@@ -1547,12 +1562,12 @@
searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
return results;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public void addLibrary(JobId jobId, Library library) throws MetadataException, RemoteException {
+ public void addLibrary(JobId jobId, Library library) throws AlgebricksException, RemoteException {
try {
// Insert into the 'Library' dataset.
LibraryTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getLibraryTupleTranslator(true);
@@ -1561,22 +1576,22 @@
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
- throw new MetadataException("A library with this name " + library.getDataverseName()
+ throw new AlgebricksException("A library with this name " + library.getDataverseName()
+ " already exists in dataverse '" + library.getDataverseName() + "'.", e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public void dropLibrary(JobId jobId, String dataverseName, String libraryName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
Library library = getLibrary(jobId, dataverseName, libraryName);
if (library == null) {
- throw new MetadataException("Cannot drop library '" + library + "' because it doesn't exist.");
+ throw new AlgebricksException("Cannot drop library '" + library + "' because it doesn't exist.");
}
try {
// Delete entry from the 'Library' dataset.
@@ -1592,19 +1607,19 @@
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS)
&& e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
- throw new MetadataException("Cannot drop library '" + libraryName, e);
+ throw new AlgebricksException("Cannot drop library '" + libraryName, e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public Library getLibrary(JobId jobId, String dataverseName, String libraryName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, libraryName);
LibraryTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getLibraryTupleTranslator(false);
@@ -1616,17 +1631,17 @@
}
return results.get(0);
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public int getMostRecentDatasetId() throws MetadataException, RemoteException {
+ public int getMostRecentDatasetId() throws AlgebricksException, RemoteException {
return DatasetIdFactory.getMostRecentDatasetId();
}
@Override
- public void addFeedPolicy(JobId jobId, FeedPolicyEntity feedPolicy) throws MetadataException, RemoteException {
+ public void addFeedPolicy(JobId jobId, FeedPolicyEntity feedPolicy) throws AlgebricksException, RemoteException {
try {
// Insert into the 'FeedPolicy' dataset.
FeedPolicyTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedPolicyTupleTranslator(true);
@@ -1634,19 +1649,19 @@
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, feedPolicyTuple);
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
- throw new MetadataException("A feed policy with this name " + feedPolicy.getPolicyName()
+ throw new AlgebricksException("A feed policy with this name " + feedPolicy.getPolicyName()
+ " already exists in dataverse '" + feedPolicy.getPolicyName() + "'.", e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public FeedPolicyEntity getFeedPolicy(JobId jobId, String dataverse, String policyName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverse, policyName);
FeedPolicyTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedPolicyTupleTranslator(false);
@@ -1658,24 +1673,24 @@
}
return null;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public void addFeedConnection(JobId jobId, FeedConnection feedConnection) throws MetadataException {
+ public void addFeedConnection(JobId jobId, FeedConnection feedConnection) throws AlgebricksException {
try {
FeedConnectionTupleTranslator tupleReaderWriter = new FeedConnectionTupleTranslator(true);
ITupleReference feedConnTuple = tupleReaderWriter.getTupleFromMetadataEntity(feedConnection);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, feedConnTuple);
} catch (HyracksDataException | ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public List<FeedConnection> getFeedConnections(JobId jobId, String dataverseName, String feedName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, feedName);
FeedConnectionTupleTranslator tupleReaderWriter = new FeedConnectionTupleTranslator(false);
@@ -1684,13 +1699,13 @@
searchIndex(jobId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, searchKey, valueExtractor, results);
return results;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public FeedConnection getFeedConnection(JobId jobId, String dataverseName, String feedName, String datasetName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, feedName, datasetName);
FeedConnectionTupleTranslator tupleReaderWriter = new FeedConnectionTupleTranslator(false);
@@ -1702,25 +1717,25 @@
}
return null;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public void dropFeedConnection(JobId jobId, String dataverseName, String feedName, String datasetName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, feedName, datasetName);
ITupleReference tuple =
getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, searchKey);
deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, tuple);
} catch (HyracksDataException | ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public void addFeed(JobId jobId, Feed feed) throws MetadataException, RemoteException {
+ public void addFeed(JobId jobId, Feed feed) throws AlgebricksException, RemoteException {
try {
// Insert into the 'Feed' dataset.
FeedTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedTupleTranslator(true);
@@ -1728,18 +1743,18 @@
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, feedTuple);
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
- throw new MetadataException("A feed with this name " + feed.getFeedName()
+ throw new AlgebricksException("A feed with this name " + feed.getFeedName()
+ " already exists in dataverse '" + feed.getDataverseName() + "'.", e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public Feed getFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException {
+ public Feed getFeed(JobId jobId, String dataverse, String feedName) throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverse, feedName);
FeedTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedTupleTranslator(false);
@@ -1751,12 +1766,12 @@
}
return null;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public List<Feed> getFeeds(JobId jobId, String dataverse) throws MetadataException, RemoteException {
+ public List<Feed> getFeeds(JobId jobId, String dataverse) throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverse);
FeedTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedTupleTranslator(false);
@@ -1765,12 +1780,12 @@
searchIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, searchKey, valueExtractor, results);
return results;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public void dropFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException {
+ public void dropFeed(JobId jobId, String dataverse, String feedName) throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverse, feedName);
// Searches the index for the tuple to be deleted. Acquires an S
@@ -1782,18 +1797,18 @@
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS)
&& e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
- throw new MetadataException("Cannot drop feed '" + feedName + "' because it doesn't exist", e);
+ throw new AlgebricksException("Cannot drop feed '" + feedName + "' because it doesn't exist", e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public void dropFeedPolicy(JobId jobId, String dataverseName, String policyName)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverseName, policyName);
ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, searchKey);
@@ -1801,18 +1816,18 @@
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS)
&& e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
- throw new MetadataException("Unknown feed policy " + policyName, e);
+ throw new AlgebricksException("Unknown feed policy " + policyName, e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public List<FeedPolicyEntity> getDataversePolicies(JobId jobId, String dataverse)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataverse);
FeedPolicyTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFeedPolicyTupleTranslator(false);
@@ -1821,12 +1836,12 @@
searchIndex(jobId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, searchKey, valueExtractor, results);
return results;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public void addExternalFile(JobId jobId, ExternalFile externalFile) throws MetadataException, RemoteException {
+ public void addExternalFile(JobId jobId, ExternalFile externalFile) throws AlgebricksException, RemoteException {
try {
// Insert into the 'externalFiles' dataset.
ExternalFileTupleTranslator tupleReaderWriter =
@@ -1835,19 +1850,20 @@
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, externalFileTuple);
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
- throw new MetadataException("An externalFile with this number " + externalFile.getFileNumber()
+ throw new AlgebricksException("An externalFile with this number " + externalFile.getFileNumber()
+ " already exists in dataset '" + externalFile.getDatasetName() + "' in dataverse '"
+ externalFile.getDataverseName() + "'.", e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public List<ExternalFile> getExternalFiles(JobId jobId, Dataset dataset) throws MetadataException, RemoteException {
+ public List<ExternalFile> getExternalFiles(JobId jobId, Dataset dataset)
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createTuple(dataset.getDataverseName(), dataset.getDatasetName());
ExternalFileTupleTranslator tupleReaderWriter =
@@ -1857,13 +1873,13 @@
searchIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey, valueExtractor, results);
return results;
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
public void dropExternalFile(JobId jobId, String dataverseName, String datasetName, int fileNumber)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
// Delete entry from the 'ExternalFile' dataset.
ITupleReference searchKey = createExternalFileSearchTuple(dataverseName, datasetName, fileNumber);
@@ -1875,26 +1891,27 @@
} catch (HyracksDataException e) {
if (e.getComponent().equals(ErrorCode.HYRACKS)
&& e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
- throw new MetadataException("Couldn't drop externalFile.", e);
+ throw new AlgebricksException("Couldn't drop externalFile.", e);
} else {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
} catch (ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public void dropExternalFiles(JobId jobId, Dataset dataset) throws MetadataException, RemoteException {
+ public void dropExternalFiles(JobId jobId, Dataset dataset) throws AlgebricksException, RemoteException {
List<ExternalFile> files = getExternalFiles(jobId, dataset);
- //loop through files and delete them
+ // loop through files and delete them
for (int i = 0; i < files.size(); i++) {
dropExternalFile(jobId, files.get(i).getDataverseName(), files.get(i).getDatasetName(),
files.get(i).getFileNumber());
}
}
- // This method is used to create a search tuple for external data file since the search tuple has an int value
+ // This method is used to create a search tuple for external data file since the
+ // search tuple has an int value
@SuppressWarnings("unchecked")
public ITupleReference createExternalFileSearchTuple(String dataverseName, String datasetName, int fileNumber)
throws HyracksDataException {
@@ -1906,17 +1923,17 @@
AMutableString aString = new AMutableString("");
ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(3);
- //dataverse field
+ // dataverse field
aString.setValue(dataverseName);
stringSerde.serialize(aString, tupleBuilder.getDataOutput());
tupleBuilder.addFieldEndOffset();
- //dataset field
+ // dataset field
aString.setValue(datasetName);
stringSerde.serialize(aString, tupleBuilder.getDataOutput());
tupleBuilder.addFieldEndOffset();
- //file number field
+ // file number field
intSerde.serialize(new AInt32(fileNumber), tupleBuilder.getDataOutput());
tupleBuilder.addFieldEndOffset();
@@ -1927,7 +1944,7 @@
@Override
public ExternalFile getExternalFile(JobId jobId, String dataverseName, String datasetName, Integer fileNumber)
- throws MetadataException, RemoteException {
+ throws AlgebricksException, RemoteException {
try {
ITupleReference searchKey = createExternalFileSearchTuple(dataverseName, datasetName, fileNumber);
ExternalFileTupleTranslator tupleReaderWriter =
@@ -1940,12 +1957,12 @@
}
return results.get(0);
} catch (HyracksDataException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
@Override
- public void updateDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException {
+ public void updateDataset(JobId jobId, Dataset dataset) throws AlgebricksException, RemoteException {
try {
// This method will delete previous entry of the dataset and insert the new one
// Delete entry from the 'datasets' dataset.
@@ -1962,7 +1979,7 @@
datasetTuple = tupleReaderWriter.getTupleFromMetadataEntity(dataset);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
} catch (HyracksDataException | ACIDException e) {
- throw new MetadataException(e);
+ throw new AlgebricksException(e);
}
}
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntityTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntityTupleTranslator.java
index fbeb353..cd84256 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntityTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntityTupleTranslator.java
@@ -19,11 +19,10 @@
package org.apache.asterix.metadata.api;
-import java.io.IOException;
import java.io.Serializable;
import java.rmi.RemoteException;
-import org.apache.asterix.common.exceptions.MetadataException;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -36,19 +35,19 @@
public interface IMetadataEntityTupleTranslator<T> extends Serializable {
/**
- * Transforms a metadata entity of type T from a given tuple to a Java
- * object (deserializing the appropriate field(s) in the tuple as
- * necessary).
+ * Transforms a metadata entity of type T from a given tuple to a Java object
+ * (deserializing the appropriate field(s) in the tuple as necessary).
*
* @param tuple
- * Tuple containing a serialized representation of a metadata
- * entity of type T.
+ * Tuple containing a serialized representation of a metadata entity
+ * of type T.
* @return A new instance of a metadata entity of type T.
- * @throws MetadataException
- * @throws IOException
+ * @throws AlgebricksException
+ * @throws HyracksDataException
+ * @throws RemoteException
*/
- public T getMetadataEntityFromTuple(ITupleReference tuple)
- throws MetadataException, HyracksDataException, RemoteException;
+ T getMetadataEntityFromTuple(ITupleReference tuple)
+ throws AlgebricksException, HyracksDataException, RemoteException;
/**
* Serializes the given metadata entity of type T into an appropriate tuple
@@ -56,7 +55,8 @@
*
* @param metadataEntity
* Metadata entity to be written into a tuple.
+ * @throws AlgebricksException
* @throws HyracksDataException
*/
- public ITupleReference getTupleFromMetadataEntity(T metadataEntity) throws MetadataException, HyracksDataException;
+ ITupleReference getTupleFromMetadataEntity(T metadataEntity) throws AlgebricksException, HyracksDataException;
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
index 88153a3..43a927b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
@@ -23,7 +23,6 @@
import java.util.List;
import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.common.metadata.IMetadataBootstrap;
import org.apache.asterix.external.indexing.ExternalFile;
@@ -41,6 +40,7 @@
import org.apache.asterix.metadata.entities.Library;
import org.apache.asterix.metadata.entities.Node;
import org.apache.asterix.metadata.entities.NodeGroup;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
/**
* A metadata manager provides user access to Asterix metadata (e.g., types,
@@ -86,8 +86,8 @@
void abortTransaction(MetadataTransactionContext ctx) throws ACIDException, RemoteException;
/**
- * Locks the metadata in given mode. The lock acquisition is delegated to
- * the metadata node. This method blocks until the lock can be acquired.
+ * Locks the metadata in given mode. The lock acquisition is delegated to the
+ * metadata node. This method blocks until the lock can be acquired.
*
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
@@ -115,10 +115,10 @@
* MetadataTransactionContext of an active metadata transaction.
* @param dataverse
* Dataverse instance to be inserted.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataverse already exists.
*/
- void addDataverse(MetadataTransactionContext ctx, Dataverse dataverse) throws MetadataException;
+ void addDataverse(MetadataTransactionContext ctx, Dataverse dataverse) throws AlgebricksException;
/**
* Retrieves all dataverses
@@ -126,9 +126,9 @@
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @return A list of dataverse instances.
- * @throws MetadataException
+ * @throws AlgebricksException
*/
- List<Dataverse> getDataverses(MetadataTransactionContext ctx) throws MetadataException;
+ List<Dataverse> getDataverses(MetadataTransactionContext ctx) throws AlgebricksException;
/**
* Retrieves a dataverse with given name.
@@ -138,10 +138,10 @@
* @param dataverseName
* Name of the dataverse to retrieve.
* @return A dataverse instance.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataverse does not exist.
*/
- Dataverse getDataverse(MetadataTransactionContext ctx, String dataverseName) throws MetadataException;
+ Dataverse getDataverse(MetadataTransactionContext ctx, String dataverseName) throws AlgebricksException;
/**
* Retrieves all datasets belonging to the given dataverse.
@@ -151,10 +151,10 @@
* @param dataverseName
* Name of the dataverse of which to find all datasets.
* @return A list of dataset instances.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataverse does not exist.
*/
- List<Dataset> getDataverseDatasets(MetadataTransactionContext ctx, String dataverseName) throws MetadataException;
+ List<Dataset> getDataverseDatasets(MetadataTransactionContext ctx, String dataverseName) throws AlgebricksException;
/**
* Deletes the dataverse with given name, and all it's associated datasets,
@@ -163,10 +163,10 @@
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @return A list of dataset instances.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataverse does not exist.
*/
- void dropDataverse(MetadataTransactionContext ctx, String dataverseName) throws MetadataException;
+ void dropDataverse(MetadataTransactionContext ctx, String dataverseName) throws AlgebricksException;
/**
* Inserts a new dataset into the metadata.
@@ -175,10 +175,10 @@
* MetadataTransactionContext of an active metadata transaction.
* @param dataset
* Dataset instance to be inserted.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataset already exists.
*/
- void addDataset(MetadataTransactionContext ctx, Dataset dataset) throws MetadataException;
+ void addDataset(MetadataTransactionContext ctx, Dataset dataset) throws AlgebricksException;
/**
* Retrieves a dataset within a given dataverse.
@@ -190,11 +190,11 @@
* @param datasetName
* Name of dataset to be retrieved.
* @return A dataset instance.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataset does not exist.
*/
Dataset getDataset(MetadataTransactionContext ctx, String dataverseName, String datasetName)
- throws MetadataException;
+ throws AlgebricksException;
/**
* Retrieves all indexes of a dataset.
@@ -206,11 +206,11 @@
* @param datasetName
* Name of dataset for which to retrieve all indexes.
* @return A list of Index instances.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataset and/or dataverse does not exist.
*/
List<Index> getDatasetIndexes(MetadataTransactionContext ctx, String dataverseName, String datasetName)
- throws MetadataException;
+ throws AlgebricksException;
/**
* Deletes the dataset with given name, and all it's associated indexes.
@@ -221,10 +221,11 @@
* Name of dataverse which holds the given dataset.
* @param datasetName
* Name of dataset to delete.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataset and/or dataverse does not exist.
*/
- void dropDataset(MetadataTransactionContext ctx, String dataverseName, String datasetName) throws MetadataException;
+ void dropDataset(MetadataTransactionContext ctx, String dataverseName, String datasetName)
+ throws AlgebricksException;
/**
* Inserts an index into the metadata. The index itself knows its name, and
@@ -234,10 +235,10 @@
* MetadataTransactionContext of an active metadata transaction.
* @param index
* Index instance to be inserted.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the index already exists.
*/
- void addIndex(MetadataTransactionContext ctx, Index index) throws MetadataException;
+ void addIndex(MetadataTransactionContext ctx, Index index) throws AlgebricksException;
/**
* Retrieves the index with given name, in given dataverse and dataset.
@@ -251,11 +252,11 @@
* @param indexName
* Name of the index to retrieve.
* @return An Index instance.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the index does not exist.
*/
Index getIndex(MetadataTransactionContext ctx, String dataverseName, String datasetName, String indexName)
- throws MetadataException;
+ throws AlgebricksException;
/**
* Deletes the index with given name, in given dataverse and dataset.
@@ -268,11 +269,11 @@
* Name of the dataset holding the index.
* @param indexName
* Name of the index to retrieve.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the index does not exist.
*/
void dropIndex(MetadataTransactionContext ctx, String dataverseName, String datasetName, String indexName)
- throws MetadataException;
+ throws AlgebricksException;
/**
* Inserts a datatype.
@@ -281,10 +282,10 @@
* MetadataTransactionContext of an active metadata transaction.
* @param datatype
* Datatype instance to be inserted.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the datatype already exists.
*/
- void addDatatype(MetadataTransactionContext ctx, Datatype datatype) throws MetadataException;
+ void addDatatype(MetadataTransactionContext ctx, Datatype datatype) throws AlgebricksException;
/**
* Retrieves the datatype with given name in given dataverse.
@@ -296,11 +297,11 @@
* @param datatypeName
* Name of datatype to be retrieved.
* @return A datatype instance.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the datatype does not exist.
*/
Datatype getDatatype(MetadataTransactionContext ctx, String dataverseName, String datatypeName)
- throws MetadataException;
+ throws AlgebricksException;
/**
* Deletes the given datatype in given dataverse.
@@ -311,12 +312,12 @@
* Name of dataverse holding the datatype.
* @param datatypeName
* Name of datatype to be deleted.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if there are still datasets using the type to be
* deleted.
*/
void dropDatatype(MetadataTransactionContext ctx, String dataverseName, String datatypeName)
- throws MetadataException;
+ throws AlgebricksException;
/**
* Inserts a node group.
@@ -325,10 +326,10 @@
* MetadataTransactionContext of an active metadata transaction.
* @param nodeGroup
* Node group instance to insert.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the node group already exists.
*/
- void addNodegroup(MetadataTransactionContext ctx, NodeGroup nodeGroup) throws MetadataException;
+ void addNodegroup(MetadataTransactionContext ctx, NodeGroup nodeGroup) throws AlgebricksException;
/**
* Retrieves a node group.
@@ -337,10 +338,10 @@
* MetadataTransactionContext of an active metadata transaction.
* @param nodeGroupName
* Name of node group to be retrieved.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the node group does not exist.
*/
- NodeGroup getNodegroup(MetadataTransactionContext ctx, String nodeGroupName) throws MetadataException;
+ NodeGroup getNodegroup(MetadataTransactionContext ctx, String nodeGroupName) throws AlgebricksException;
/**
* Deletes a node group.
@@ -350,13 +351,14 @@
* @param nodeGroupName
* Name of node group to be deleted.
* @param failSilently
- * true means it's a no-op if the node group cannot be dropped; false means it will throw an exception.
- * @throws MetadataException
+ * true means it's a no-op if the node group cannot be dropped; false
+ * means it will throw an exception.
+ * @throws AlgebricksException
* For example, there are still datasets partitioned on the node
* group to be deleted.
*/
void dropNodegroup(MetadataTransactionContext ctx, String nodeGroupName, boolean failSilently)
- throws MetadataException;
+ throws AlgebricksException;
/**
* Inserts a node (machine).
@@ -365,20 +367,20 @@
* MetadataTransactionContext of an active metadata transaction.
* @param node
* Node instance to be inserted.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the node already exists.
*/
- void addNode(MetadataTransactionContext ctx, Node node) throws MetadataException;
+ void addNode(MetadataTransactionContext ctx, Node node) throws AlgebricksException;
/**
* @param mdTxnCtx
* MetadataTransactionContext of an active metadata transaction.
* @param function
- * An instance of type Function that represents the function
- * being added
- * @throws MetadataException
+ * An instance of type Function that represents the function being
+ * added
+ * @throws AlgebricksException
*/
- void addFunction(MetadataTransactionContext mdTxnCtx, Function function) throws MetadataException;
+ void addFunction(MetadataTransactionContext mdTxnCtx, Function function) throws AlgebricksException;
/**
* @param ctx
@@ -386,21 +388,22 @@
* @param functionSignature
* the functions signature (unique to the function)
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
*/
- Function getFunction(MetadataTransactionContext ctx, FunctionSignature functionSignature) throws MetadataException;
+ Function getFunction(MetadataTransactionContext ctx, FunctionSignature functionSignature)
+ throws AlgebricksException;
- List<Function> getFunctions(MetadataTransactionContext ctx, String dataverseName) throws MetadataException;
+ List<Function> getFunctions(MetadataTransactionContext ctx, String dataverseName) throws AlgebricksException;
/**
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param functionSignature
* the functions signature (unique to the function)
- * @throws MetadataException
+ * @throws AlgebricksException
*/
- void dropFunction(MetadataTransactionContext ctx, FunctionSignature functionSignature) throws MetadataException;
+ void dropFunction(MetadataTransactionContext ctx, FunctionSignature functionSignature) throws AlgebricksException;
/**
* @param mdTxnCtx
@@ -408,9 +411,9 @@
* @param adapter
* An instance of type Adapter that represents the adapter being
* added
- * @throws MetadataException
+ * @throws AlgebricksException
*/
- void addAdapter(MetadataTransactionContext mdTxnCtx, DatasourceAdapter adapter) throws MetadataException;
+ void addAdapter(MetadataTransactionContext mdTxnCtx, DatasourceAdapter adapter) throws AlgebricksException;
/**
* @param ctx
@@ -420,10 +423,10 @@
* @param name
* name of the adapter
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
*/
DatasourceAdapter getAdapter(MetadataTransactionContext ctx, String dataverseName, String name)
- throws MetadataException;
+ throws AlgebricksException;
/**
* @param ctx
@@ -432,9 +435,9 @@
* the dataverse associated with the adapter being dropped
* @param name
* name of the adapter
- * @throws MetadataException
+ * @throws AlgebricksException
*/
- void dropAdapter(MetadataTransactionContext ctx, String dataverseName, String name) throws MetadataException;
+ void dropAdapter(MetadataTransactionContext ctx, String dataverseName, String name) throws AlgebricksException;
/**
* @param ctx
@@ -442,90 +445,91 @@
* @param dataverseName
* the dataverse whose associated adapters are being requested
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
*/
List<DatasourceAdapter> getDataverseAdapters(MetadataTransactionContext ctx, String dataverseName)
- throws MetadataException;
+ throws AlgebricksException;
/**
* @param ctx
* @param policy
- * @throws MetadataException
+ * @throws AlgebricksException
*/
- void addCompactionPolicy(MetadataTransactionContext ctx, CompactionPolicy policy) throws MetadataException;
+ void addCompactionPolicy(MetadataTransactionContext ctx, CompactionPolicy policy) throws AlgebricksException;
/**
* @param ctx
* @param dataverse
* @param policyName
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
*/
CompactionPolicy getCompactionPolicy(MetadataTransactionContext ctx, String dataverse, String policyName)
- throws MetadataException;
+ throws AlgebricksException;
/**
* @param ctx
* @param dataverseName
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
*/
- List<Function> getDataverseFunctions(MetadataTransactionContext ctx, String dataverseName) throws MetadataException;
+ List<Function> getDataverseFunctions(MetadataTransactionContext ctx, String dataverseName)
+ throws AlgebricksException;
/**
* @param ctx
* @param feed
- * @throws MetadataException
+ * @throws AlgebricksException
*/
- void addFeed(MetadataTransactionContext ctx, Feed feed) throws MetadataException;
+ void addFeed(MetadataTransactionContext ctx, Feed feed) throws AlgebricksException;
/**
* @param ctx
* @param dataverse
* @param feedName
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
*/
- Feed getFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException;
+ Feed getFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws AlgebricksException;
- List<Feed> getFeeds(MetadataTransactionContext ctx, String dataverse) throws MetadataException;
+ List<Feed> getFeeds(MetadataTransactionContext ctx, String dataverse) throws AlgebricksException;
/**
* @param ctx
* @param dataverse
* @param feedName
- * @throws MetadataException
+ * @throws AlgebricksException
*/
- void dropFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException;
+ void dropFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws AlgebricksException;
/**
* @param ctx
* @param policy
- * @throws MetadataException
+ * @throws AlgebricksException
*/
- void addFeedPolicy(MetadataTransactionContext ctx, FeedPolicyEntity policy) throws MetadataException;
+ void addFeedPolicy(MetadataTransactionContext ctx, FeedPolicyEntity policy) throws AlgebricksException;
/**
* @param ctx
* @param dataverse
* @param policyName
- * @throws MetadataException
+ * @throws AlgebricksException
*/
- void dropFeedPolicy(MetadataTransactionContext ctx, String dataverse, String policyName) throws MetadataException;
+ void dropFeedPolicy(MetadataTransactionContext ctx, String dataverse, String policyName) throws AlgebricksException;
/**
* @param ctx
* @param dataverse
* @param policyName
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
*/
FeedPolicyEntity getFeedPolicy(MetadataTransactionContext ctx, String dataverse, String policyName)
- throws MetadataException;
+ throws AlgebricksException;
- void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws MetadataException;
+ void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws AlgebricksException;
- int getMostRecentDatasetId() throws MetadataException;
+ int getMostRecentDatasetId() throws AlgebricksException;
void acquireWriteLatch();
@@ -536,32 +540,32 @@
void releaseReadLatch();
/**
- * Removes a library , acquiring local locks on behalf of the given
- * transaction id.
+ * Removes a library , acquiring local locks on behalf of the given transaction
+ * id.
*
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param dataverseName
* dataverse asociated with the adapter that is to be deleted.
* @param libraryName
- * Name of library to be deleted. MetadataException for example,
- * if the library does not exists.
- * @throws MetadataException
+ * Name of library to be deleted. AlgebricksException for example, if
+ * the library does not exists.
+ * @throws AlgebricksException
*/
- void dropLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName) throws MetadataException;
+ void dropLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName)
+ throws AlgebricksException;
/**
- * Adds a library, acquiring local locks on behalf of the given
- * transaction id.
+ * Adds a library, acquiring local locks on behalf of the given transaction id.
*
* @param ctx
* MetadataTransactionContext of an active metadata transaction.
* @param library
* Library to be added
- * @throws MetadataException
+ * @throws AlgebricksException
* for example, if the library is already added.
*/
- void addLibrary(MetadataTransactionContext ctx, Library library) throws MetadataException;
+ void addLibrary(MetadataTransactionContext ctx, Library library) throws AlgebricksException;
/**
* @param ctx
@@ -571,11 +575,11 @@
* @param libraryName
* name of the library that is to be retrieved
* @return Library
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
Library getLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
/**
* Retireve libraries installed in a given dataverse.
@@ -585,19 +589,20 @@
* @param dataverseName
* dataverse asociated with the library that is to be retrieved.
* @return Library
- * @throws MetadataException
+ * @throws AlgebricksException
*/
- List<Library> getDataverseLibraries(MetadataTransactionContext ctx, String dataverseName) throws MetadataException;
+ List<Library> getDataverseLibraries(MetadataTransactionContext ctx, String dataverseName)
+ throws AlgebricksException;
/**
* @param mdTxnCtx
* MetadataTransactionContext of an active metadata transaction.
* @param externalFile
- * An instance of type ExternalFile that represents the external file being
- * added
- * @throws MetadataException
+ * An instance of type ExternalFile that represents the external file
+ * being added
+ * @throws AlgebricksException
*/
- void addExternalFile(MetadataTransactionContext mdTxnCtx, ExternalFile externalFile) throws MetadataException;
+ void addExternalFile(MetadataTransactionContext mdTxnCtx, ExternalFile externalFile) throws AlgebricksException;
/**
* @param mdTxnCtx
@@ -605,20 +610,20 @@
* @param dataset
* An instance of type Dataset that represents the "external" dataset
* @return A list of external files belonging to the dataset
- * @throws MetadataException
+ * @throws AlgebricksException
*/
List<ExternalFile> getDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset)
- throws MetadataException;
+ throws AlgebricksException;
/**
* @param mdTxnCtx
* MetadataTransactionContext of an active metadata transaction.
* @param externalFile
- * An instance of type ExternalFile that represents the external file being
- * dropped
- * @throws MetadataException
+ * An instance of type ExternalFile that represents the external file
+ * being dropped
+ * @throws AlgebricksException
*/
- void dropExternalFile(MetadataTransactionContext mdTxnCtx, ExternalFile externalFile) throws MetadataException;
+ void dropExternalFile(MetadataTransactionContext mdTxnCtx, ExternalFile externalFile) throws AlgebricksException;
/**
* @param mdTxnCtx
@@ -626,9 +631,9 @@
* @param dataset
* An instance of type dataset that owns the external files being
* dropped
- * @throws MetadataException
+ * @throws AlgebricksException
*/
- void dropDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset) throws MetadataException;
+ void dropDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset) throws AlgebricksException;
/**
* Get en external file
@@ -638,10 +643,10 @@
* @param datasetName
* @param fileNumber
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
*/
ExternalFile getExternalFile(MetadataTransactionContext mdTxnCtx, String dataverseName, String datasetName,
- Integer fileNumber) throws MetadataException;
+ Integer fileNumber) throws AlgebricksException;
/**
* update an existing dataset in metadata.
@@ -650,76 +655,80 @@
* MetadataTransactionContext of an active metadata transaction.
* @param dataset
* Existing Dataset.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataset already exists.
*/
- void updateDataset(MetadataTransactionContext ctx, Dataset dataset) throws MetadataException;
+ void updateDataset(MetadataTransactionContext ctx, Dataset dataset) throws AlgebricksException;
/**
* Clean up temporary datasets that have not been active for a long time.
*
- * @throws MetadataException
+ * @throws AlgebricksException
*/
- void cleanupTempDatasets() throws MetadataException;
+ void cleanupTempDatasets() throws AlgebricksException;
/**
- * Add an extension entity to its extension dataset under the ongoing metadata transaction
+ * Add an extension entity to its extension dataset under the ongoing metadata
+ * transaction
*
* @param mdTxnCtx
* @param entity
- * @throws MetadataException
+ * @throws AlgebricksException
*/
<T extends IExtensionMetadataEntity> void addEntity(MetadataTransactionContext mdTxnCtx, T entity)
- throws MetadataException;
+ throws AlgebricksException;
/**
- * Upsert an extension entity to its extension dataset under the ongoing metadata transaction
+ * Upsert an extension entity to its extension dataset under the ongoing
+ * metadata transaction
*
* @param mdTxnCtx
* @param entity
- * @throws MetadataException
+ * @throws AlgebricksException
*/
<T extends IExtensionMetadataEntity> void upsertEntity(MetadataTransactionContext mdTxnCtx, T entity)
- throws MetadataException;
+ throws AlgebricksException;
/**
- * Deletes an extension entity from its extension dataset under the ongoing metadata transaction
+ * Deletes an extension entity from its extension dataset under the ongoing
+ * metadata transaction
*
* @param mdTxnCtx
* @param entity
- * @throws MetadataException
+ * @throws AlgebricksException
*/
<T extends IExtensionMetadataEntity> void deleteEntity(MetadataTransactionContext mdTxnCtx, T entity)
- throws MetadataException;
+ throws AlgebricksException;
/**
- * Gets a list of extension entities matching a search key under the ongoing metadata transaction
+ * Gets a list of extension entities matching a search key under the ongoing
+ * metadata transaction
*
* @param mdTxnCtx
* @param searchKey
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
*/
<T extends IExtensionMetadataEntity> List<T> getEntities(MetadataTransactionContext mdTxnCtx,
- IExtensionMetadataSearchKey searchKey) throws MetadataException;
+ IExtensionMetadataSearchKey searchKey) throws AlgebricksException;
/**
- * Indicate when the metadata node has left or rejoined the cluster, and the MetadataManager should
- * rebind it
+ * Indicate when the metadata node has left or rejoined the cluster, and the
+ * MetadataManager should rebind it
*/
void rebindMetadataNode();
/**
* Feed Connection Related Metadata operations
*/
- void addFeedConnection(MetadataTransactionContext ctx, FeedConnection feedConnection) throws MetadataException;
+ void addFeedConnection(MetadataTransactionContext ctx, FeedConnection feedConnection) throws AlgebricksException;
void dropFeedConnection(MetadataTransactionContext ctx, String dataverseName, String feedName, String datasetName)
- throws MetadataException;
+ throws AlgebricksException;
FeedConnection getFeedConnection(MetadataTransactionContext ctx, String dataverseName, String feedName,
- String datasetName) throws MetadataException;
+ String datasetName) throws AlgebricksException;
List<FeedConnection> getFeedConections(MetadataTransactionContext ctx, String dataverseName, String feedName)
- throws MetadataException;
+ throws AlgebricksException;
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
index a990d29..8c4920f 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
@@ -25,7 +25,6 @@
import java.util.List;
import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.common.transactions.JobId;
import org.apache.asterix.external.indexing.ExternalFile;
@@ -42,15 +41,15 @@
import org.apache.asterix.metadata.entities.Library;
import org.apache.asterix.metadata.entities.Node;
import org.apache.asterix.metadata.entities.NodeGroup;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
/**
* A metadata node stores metadata in its local storage structures (currently
* BTrees). A metadata node services requests on behalf of the (possibly remote)
* metadata manager by executing local transactions against its local storage.
- * This interface mirrors the methods in IMetadataManager.
- * Users wanting to lock/access metadata shall always go through the
- * MetadataManager, and should never call methods on the MetadataNode directly
- * for any reason.
+ * This interface mirrors the methods in IMetadataManager. Users wanting to
+ * lock/access metadata shall always go through the MetadataManager, and should
+ * never call methods on the MetadataNode directly for any reason.
*/
public interface IMetadataNode extends Remote, Serializable {
@@ -96,46 +95,46 @@
void unlock(JobId jobId, byte lockMode) throws ACIDException, RemoteException;
/**
- * Inserts a new dataverse into the metadata, acquiring local locks on
- * behalf of the given transaction id.
+ * Inserts a new dataverse into the metadata, acquiring local locks on behalf of
+ * the given transaction id.
*
* @param jobId
* A globally unique id for an active metadata transaction.
* @param dataverse
* Dataverse instance to be inserted.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataverse already exists.
* @throws RemoteException
*/
- void addDataverse(JobId jobId, Dataverse dataverse) throws MetadataException, RemoteException;
+ void addDataverse(JobId jobId, Dataverse dataverse) throws AlgebricksException, RemoteException;
/**
- * Retrieves all dataverses, acquiring local locks on behalf of
- * the given transaction id.
+ * Retrieves all dataverses, acquiring local locks on behalf of the given
+ * transaction id.
*
* @param jobId
* A globally unique id for an active metadata transaction.
* @return A list of dataverse instances.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataverse does not exist.
* @throws RemoteException
*/
- List<Dataverse> getDataverses(JobId jobId) throws MetadataException, RemoteException;
+ List<Dataverse> getDataverses(JobId jobId) throws AlgebricksException, RemoteException;
/**
- * Retrieves a dataverse with given name, acquiring local locks on behalf of
- * the given transaction id.
+ * Retrieves a dataverse with given name, acquiring local locks on behalf of the
+ * given transaction id.
*
* @param jobId
* A globally unique id for an active metadata transaction.
* @param dataverseName
* Name of the dataverse to retrieve.
* @return A dataverse instance.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataverse does not exist.
* @throws RemoteException
*/
- Dataverse getDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
+ Dataverse getDataverse(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException;
/**
* Retrieves all datasets belonging to the given dataverse, acquiring local
@@ -146,42 +145,42 @@
* @param dataverseName
* Name of the dataverse of which to find all datasets.
* @return A list of dataset instances.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataverse does not exist. RemoteException
*/
- List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
+ List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException;
/**
* Deletes the dataverse with given name, and all it's associated datasets,
- * indexes, and types, acquiring local locks on behalf of the given
- * transaction id.
+ * indexes, and types, acquiring local locks on behalf of the given transaction
+ * id.
*
* @param jobId
* A globally unique id for an active metadata transaction.
* @return A list of dataset instances.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataverse does not exist.
* @throws RemoteException
*/
- void dropDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
+ void dropDataverse(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException;
/**
- * Inserts a new dataset into the metadata, acquiring local locks on behalf
- * of the given transaction id.
+ * Inserts a new dataset into the metadata, acquiring local locks on behalf of
+ * the given transaction id.
*
* @param jobId
* A globally unique id for an active metadata transaction.
* @param dataset
* Dataset instance to be inserted.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataset already exists.
* @throws RemoteException
*/
- void addDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException;
+ void addDataset(JobId jobId, Dataset dataset) throws AlgebricksException, RemoteException;
/**
- * Retrieves a dataset within a given dataverse, acquiring local locks on
- * behalf of the given transaction id.
+ * Retrieves a dataset within a given dataverse, acquiring local locks on behalf
+ * of the given transaction id.
*
* @param jobId
* A globally unique id for an active metadata transaction.
@@ -190,15 +189,16 @@
* @param datasetName
* Name of dataset to be retrieved.
* @return A dataset instance.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataset does not exist.
* @throws RemoteException
*/
- Dataset getDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException, RemoteException;
+ Dataset getDataset(JobId jobId, String dataverseName, String datasetName)
+ throws AlgebricksException, RemoteException;
/**
- * Retrieves all indexes of a dataset, acquiring local locks on behalf of
- * the given transaction id.
+ * Retrieves all indexes of a dataset, acquiring local locks on behalf of the
+ * given transaction id.
*
* @param jobId
* A globally unique id for an active metadata transaction.
@@ -207,12 +207,12 @@
* @param datasetName
* Name of dataset for which to retrieve all indexes.
* @return A list of Index instances.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataset and/or dataverse does not exist.
* @throws RemoteException
*/
List<Index> getDatasetIndexes(JobId jobId, String dataverseName, String datasetName)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
/**
* Deletes the dataset with given name, and all it's associated indexes,
@@ -224,26 +224,26 @@
* Name of dataverse which holds the given dataset.
* @param datasetName
* Name of dataset to delete.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataset and/or dataverse does not exist.
* @throws RemoteException
*/
- void dropDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException, RemoteException;
+ void dropDataset(JobId jobId, String dataverseName, String datasetName) throws AlgebricksException, RemoteException;
/**
- * Inserts an index into the metadata, acquiring local locks on behalf of
- * the given transaction id. The index itself knows its name, and which
- * dataset it belongs to.
+ * Inserts an index into the metadata, acquiring local locks on behalf of the
+ * given transaction id. The index itself knows its name, and which dataset it
+ * belongs to.
*
* @param jobId
* A globally unique id for an active metadata transaction.
* @param index
* Index instance to be inserted.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the index already exists.
* @throws RemoteException
*/
- void addIndex(JobId jobId, Index index) throws MetadataException, RemoteException;
+ void addIndex(JobId jobId, Index index) throws AlgebricksException, RemoteException;
/**
* Retrieves the index with given name, in given dataverse and dataset,
@@ -257,16 +257,16 @@
* Name of the dataset holding the index.
* @indexName Name of the index to retrieve.
* @return An Index instance.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the index does not exist.
* @throws RemoteException
*/
Index getIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
/**
- * Deletes the index with given name, in given dataverse and dataset,
- * acquiring local locks on behalf of the given transaction id.
+ * Deletes the index with given name, in given dataverse and dataset, acquiring
+ * local locks on behalf of the given transaction id.
*
* @param jobId
* A globally unique id for an active metadata transaction.
@@ -275,30 +275,30 @@
* @param datasetName
* Name of the dataset holding the index.
* @indexName Name of the index to retrieve.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the index does not exist.
* @throws RemoteException
*/
void dropIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
/**
- * Inserts a datatype, acquiring local locks on behalf of the given
- * transaction id.
+ * Inserts a datatype, acquiring local locks on behalf of the given transaction
+ * id.
*
* @param jobId
* A globally unique id for an active metadata transaction.
* @param datatype
* Datatype instance to be inserted.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the datatype already exists.
* @throws RemoteException
*/
- void addDatatype(JobId jobId, Datatype datatype) throws MetadataException, RemoteException;
+ void addDatatype(JobId jobId, Datatype datatype) throws AlgebricksException, RemoteException;
/**
- * Retrieves the datatype with given name in given dataverse, acquiring
- * local locks on behalf of the given transaction id.
+ * Retrieves the datatype with given name in given dataverse, acquiring local
+ * locks on behalf of the given transaction id.
*
* @param jobId
* A globally unique id for an active metadata transaction.
@@ -307,12 +307,12 @@
* @param datatypeName
* Name of datatype to be retrieved.
* @return A datatype instance.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the datatype does not exist.
* @throws RemoteException
*/
Datatype getDatatype(JobId jobId, String dataverseName, String datatypeName)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
/**
* Deletes the given datatype in given dataverse, acquiring local locks on
@@ -324,12 +324,13 @@
* Name of dataverse holding the datatype.
* @param datatypeName
* Name of datatype to be deleted.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if there are still datasets using the type to be
* deleted.
* @throws RemoteException
*/
- void dropDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException, RemoteException;
+ void dropDatatype(JobId jobId, String dataverseName, String datatypeName)
+ throws AlgebricksException, RemoteException;
/**
* Inserts a node group, acquiring local locks on behalf of the given
@@ -339,11 +340,11 @@
* A globally unique id for an active metadata transaction.
* @param nodeGroup
* Node group instance to insert.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the node group already exists.
* @throws RemoteException
*/
- void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws MetadataException, RemoteException;
+ void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws AlgebricksException, RemoteException;
/**
* Retrieves a node group, acquiring local locks on behalf of the given
@@ -353,11 +354,11 @@
* A globally unique id for an active metadata transaction.
* @param nodeGroupName
* Name of node group to be retrieved.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the node group does not exist.
* @throws RemoteException
*/
- NodeGroup getNodeGroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException;
+ NodeGroup getNodeGroup(JobId jobId, String nodeGroupName) throws AlgebricksException, RemoteException;
/**
* Deletes a node group, acquiring local locks on behalf of the given
@@ -368,29 +369,30 @@
* @param nodeGroupName
* Name of node group to be deleted.
* @param failSilently
- * true means it's a no-op if the node group cannot be dropped; false means it will throw an exception.
+ * true means it's a no-op if the node group cannot be dropped; false
+ * means it will throw an exception.
* @return Whether the node group has been successfully dropped.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, there are still datasets partitioned on the node
* group to be deleted.
* @throws RemoteException
*/
boolean dropNodegroup(JobId jobId, String nodeGroupName, boolean failSilently)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
/**
- * Inserts a node (compute node), acquiring local locks on behalf of the
- * given transaction id.
+ * Inserts a node (compute node), acquiring local locks on behalf of the given
+ * transaction id.
*
* @param jobId
* A globally unique id for an active metadata transaction.
* @param node
* Node instance to be inserted.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the node already exists.
* @throws RemoteException
*/
- void addNode(JobId jobId, Node node) throws MetadataException, RemoteException;
+ void addNode(JobId jobId, Node node) throws AlgebricksException, RemoteException;
/**
* @param jobId
@@ -398,218 +400,223 @@
* @param functionSignature
* An instance of functionSignature representing the function
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
- Function getFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException, RemoteException;
+ Function getFunction(JobId jobId, FunctionSignature functionSignature) throws AlgebricksException, RemoteException;
- List<Function> getFunctions(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
+ List<Function> getFunctions(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException;
/**
- * Deletes a function, acquiring local locks on behalf of the given
- * transaction id.
+ * Deletes a function, acquiring local locks on behalf of the given transaction
+ * id.
*
* @param jobId
* A globally unique id for an active metadata transaction.
* @param functionSignature
* An instance of functionSignature representing the function
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, there are still datasets partitioned on the node
* group to be deleted.
* @throws RemoteException
*/
- void dropFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException, RemoteException;
+ void dropFunction(JobId jobId, FunctionSignature functionSignature) throws AlgebricksException, RemoteException;
/**
* @param jobId
* A globally unique id for an active metadata transaction.
* @param function
* Function to be inserted
- * @throws MetadataException
+ * @throws AlgebricksException
* for example, if the function already exists or refers to an
* unknown function
* @throws RemoteException
*/
- void addFunction(JobId jobId, Function function) throws MetadataException, RemoteException;
+ void addFunction(JobId jobId, Function function) throws AlgebricksException, RemoteException;
/**
- * @param ctx
+ * @param jobId
* @param dataverseName
- * @return List<Function> A list containing the functions in the specified dataverse
- * @throws MetadataException
+ * @return List<Function> A list containing the functions in the specified
+ * dataverse
+ * @throws AlgebricksException
* @throws RemoteException
*/
- List<Function> getDataverseFunctions(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
+ List<Function> getDataverseFunctions(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException;
/**
- * @param ctx
+ * @param jobId
* @param dataverseName
- * @return List<Adapter> A list containing the adapters in the specified dataverse
- * @throws MetadataException
+ * @return List<Adapter> A list containing the adapters in the specified
+ * dataverse
+ * @throws AlgebricksException
* @throws RemoteException
*/
List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
/**
* @param jobId
* @param dataverseName
* @param adapterName
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
DatasourceAdapter getAdapter(JobId jobId, String dataverseName, String adapterName)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
/**
- * Deletes a adapter , acquiring local locks on behalf of the given
- * transaction id.
+ * Deletes a adapter , acquiring local locks on behalf of the given transaction
+ * id.
*
* @param jobId
* A globally unique id for an active metadata transaction.
* @param dataverseName
* dataverse asociated with the adapter that is to be deleted.
* @param adapterName
- * Name of adapter to be deleted. MetadataException for example,
- * if the adapter does not exists.
+ * Name of adapter to be deleted. AlgebricksException for example, if
+ * the adapter does not exists.
+ * @throws AlgebricksException
* @throws RemoteException
*/
- void dropAdapter(JobId jobId, String dataverseName, String adapterName) throws MetadataException, RemoteException;
+ void dropAdapter(JobId jobId, String dataverseName, String adapterName) throws AlgebricksException, RemoteException;
/**
* @param jobId
* A globally unique id for an active metadata transaction.
* @param adapter
* Adapter to be inserted
- * @throws MetadataException
+ * @throws AlgebricksException
* for example, if the adapter already exists.
* @throws RemoteException
*/
- void addAdapter(JobId jobId, DatasourceAdapter adapter) throws MetadataException, RemoteException;
+ void addAdapter(JobId jobId, DatasourceAdapter adapter) throws AlgebricksException, RemoteException;
/**
* @param jobId
* @param compactionPolicy
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
- void addCompactionPolicy(JobId jobId, CompactionPolicy compactionPolicy) throws MetadataException, RemoteException;
+ void addCompactionPolicy(JobId jobId, CompactionPolicy compactionPolicy)
+ throws AlgebricksException, RemoteException;
/**
* @param jobId
* @param dataverse
* @param policy
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
CompactionPolicy getCompactionPolicy(JobId jobId, String dataverse, String policy)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
/**
* @param jobId
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
- void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException;
+ void initializeDatasetIdFactory(JobId jobId) throws AlgebricksException, RemoteException;
/**
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
- int getMostRecentDatasetId() throws MetadataException, RemoteException;
+ int getMostRecentDatasetId() throws AlgebricksException, RemoteException;
/**
* @param jobId
* @param feed
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
- void addFeed(JobId jobId, Feed feed) throws MetadataException, RemoteException;
+ void addFeed(JobId jobId, Feed feed) throws AlgebricksException, RemoteException;
/**
* @param jobId
* @param dataverse
* @param feedName
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
- Feed getFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException;
+ Feed getFeed(JobId jobId, String dataverse, String feedName) throws AlgebricksException, RemoteException;
- List<Feed> getFeeds(JobId jobId, String dataverse) throws MetadataException, RemoteException;
+ List<Feed> getFeeds(JobId jobId, String dataverse) throws AlgebricksException, RemoteException;
/**
* @param jobId
* @param dataverse
* @param feedName
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
- void dropFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException;
+ void dropFeed(JobId jobId, String dataverse, String feedName) throws AlgebricksException, RemoteException;
/**
* @param jobId
* @param feedPolicy
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
- void addFeedPolicy(JobId jobId, FeedPolicyEntity feedPolicy) throws MetadataException, RemoteException;
+ void addFeedPolicy(JobId jobId, FeedPolicyEntity feedPolicy) throws AlgebricksException, RemoteException;
/**
* @param jobId
* @param dataverse
* @param policy
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
FeedPolicyEntity getFeedPolicy(JobId jobId, String dataverse, String policy)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
/**
- * Removes a library , acquiring local locks on behalf of the given
- * transaction id.
+ * Removes a library , acquiring local locks on behalf of the given transaction
+ * id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param dataverseName
* dataverse asociated with the adapter that is to be deleted.
* @param libraryName
- * Name of library to be deleted. MetadataException for example,
- * if the library does not exists.
+ * Name of library to be deleted. AlgebricksException for example, if
+ * the library does not exists.
+ * @throws AlgebricksException
* @throws RemoteException
*/
- void dropLibrary(JobId jobId, String dataverseName, String libraryName) throws MetadataException, RemoteException;
+ void dropLibrary(JobId jobId, String dataverseName, String libraryName) throws AlgebricksException, RemoteException;
/**
- * Adds a library, acquiring local locks on behalf of the given
- * transaction id.
+ * Adds a library, acquiring local locks on behalf of the given transaction id.
*
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param library
* Library to be added
- * @throws MetadataException
+ * @throws AlgebricksException
* for example, if the library is already added.
* @throws RemoteException
*/
- void addLibrary(JobId jobId, Library library) throws MetadataException, RemoteException;
+ void addLibrary(JobId jobId, Library library) throws AlgebricksException, RemoteException;
/**
- * @param txnId
+ * @param jobId
* A globally unique id for an active metadata transaction.
* @param dataverseName
* dataverse asociated with the library that is to be retrieved.
* @param libraryName
* name of the library that is to be retrieved
* @return Library
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
- Library getLibrary(JobId jobId, String dataverseName, String libraryName) throws MetadataException, RemoteException;
+ Library getLibrary(JobId jobId, String dataverseName, String libraryName)
+ throws AlgebricksException, RemoteException;
/**
* Retireve libraries installed in a given dataverse.
@@ -619,19 +626,19 @@
* @param dataverseName
* dataverse asociated with the library that is to be retrieved.
* @return Library
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
- List<Library> getDataverseLibraries(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
+ List<Library> getDataverseLibraries(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException;
/**
* @param jobId
* @param dataverseName
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
- List<Feed> getDataverseFeeds(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
+ List<Feed> getDataverseFeeds(JobId jobId, String dataverseName) throws AlgebricksException, RemoteException;
/**
* delete a give feed (ingestion) policy
@@ -641,40 +648,41 @@
* @param policyName
* @return
* @throws RemoteException
- * @throws MetadataException
+ * @throws AlgebricksException
*/
- void dropFeedPolicy(JobId jobId, String dataverseName, String policyName) throws MetadataException, RemoteException;
+ void dropFeedPolicy(JobId jobId, String dataverseName, String policyName)
+ throws AlgebricksException, RemoteException;
/**
* @param jobId
* @param dataverse
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
List<FeedPolicyEntity> getDataversePolicies(JobId jobId, String dataverse)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
/**
* @param jobId
* A globally unique id for an active metadata transaction.
* @param externalFile
* An object representing the external file entity
- * @throws MetadataException
+ * @throws AlgebricksException
* for example, if the file already exists.
* @throws RemoteException
*/
- void addExternalFile(JobId jobId, ExternalFile externalFile) throws MetadataException, RemoteException;
+ void addExternalFile(JobId jobId, ExternalFile externalFile) throws AlgebricksException, RemoteException;
/**
* @param jobId
* A globally unique id for an active metadata transaction.
* @param dataset
* A dataset the files belongs to.
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
- List<ExternalFile> getExternalFiles(JobId jobId, Dataset dataset) throws MetadataException, RemoteException;
+ List<ExternalFile> getExternalFiles(JobId jobId, Dataset dataset) throws AlgebricksException, RemoteException;
/**
* Deletes an externalFile , acquiring local locks on behalf of the given
@@ -683,27 +691,30 @@
* @param jobId
* A globally unique id for an active metadata transaction.
* @param dataverseName
- * dataverse asociated with the external dataset that owns the file to be deleted.
+ * dataverse asociated with the external dataset that owns the file
+ * to be deleted.
* @param datasetName
* Name of dataset owning the file to be deleted.
* @param fileNumber
* the id number for the file to be deleted
+ * @throws AlgebricksException
* @throws RemoteException
*/
void dropExternalFile(JobId jobId, String dataverseName, String datasetName, int fileNumber)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
/**
- * Deletes all external files belonging to a dataset, acquiring local locks on behalf of the given
- * transaction id.
+ * Deletes all external files belonging to a dataset, acquiring local locks on
+ * behalf of the given transaction id.
*
* @param jobId
* A globally unique id for an active metadata transaction.
* @param dataset
* An external dataset the files belong to.
+ * @throws AlgebricksException
* @throws RemoteException
*/
- void dropExternalFiles(JobId jobId, Dataset dataset) throws MetadataException, RemoteException;
+ void dropExternalFiles(JobId jobId, Dataset dataset) throws AlgebricksException, RemoteException;
/**
* Retrieves the file with given number, in given dataverse and dataset,
@@ -718,12 +729,12 @@
* @param fileNumber
* Number of the file
* @return An ExternalFile instance.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the index does not exist.
* @throws RemoteException
*/
ExternalFile getExternalFile(JobId jobId, String dataverseName, String datasetName, Integer fileNumber)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
/**
* update an existing dataset in the metadata, acquiring local locks on behalf
@@ -733,66 +744,67 @@
* A globally unique id for an active metadata transaction.
* @param dataset
* updated Dataset instance.
- * @throws MetadataException
+ * @throws AlgebricksException
* For example, if the dataset already exists.
* @throws RemoteException
*/
- void updateDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException;
+ void updateDataset(JobId jobId, Dataset dataset) throws AlgebricksException, RemoteException;
/**
* Adds an extension entity under the ongoing transaction job id
*
* @param jobId
* @param entity
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
<T extends IExtensionMetadataEntity> void addEntity(JobId jobId, T entity)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
/**
* Upserts an extension entity under the ongoing transaction job id
*
* @param jobId
* @param entity
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
<T extends IExtensionMetadataEntity> void upsertEntity(JobId jobId, T entity)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
/**
* Deletes an extension entity under the ongoing transaction job id
*
* @param jobId
* @param entity
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
<T extends IExtensionMetadataEntity> void deleteEntity(JobId jobId, T entity)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
/**
- * Gets a list of extension entities matching a search key under the ongoing transaction
+ * Gets a list of extension entities matching a search key under the ongoing
+ * transaction
*
* @param jobId
* @param searchKey
* @return
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws RemoteException
*/
<T extends IExtensionMetadataEntity> List<T> getEntities(JobId jobId, IExtensionMetadataSearchKey searchKey)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
- void addFeedConnection(JobId jobId, FeedConnection feedConnection) throws MetadataException, RemoteException;
+ void addFeedConnection(JobId jobId, FeedConnection feedConnection) throws AlgebricksException, RemoteException;
FeedConnection getFeedConnection(JobId jobId, String dataverseName, String feedName, String datasetName)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
void dropFeedConnection(JobId jobId, String dataverseName, String feedName, String datasetName)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
List<FeedConnection> getFeedConnections(JobId jobId, String dataverseName, String feedName)
- throws MetadataException, RemoteException;
+ throws AlgebricksException, RemoteException;
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IValueExtractor.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IValueExtractor.java
index c2b61ee..7d19b20 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IValueExtractor.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IValueExtractor.java
@@ -22,8 +22,8 @@
import java.io.IOException;
import java.rmi.RemoteException;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.common.transactions.JobId;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -43,10 +43,9 @@
* @param tuple
* Tuple from which an object shall be extracted.
* @return New object of type T.
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws HyracksDataException
* @throws IOException
*/
- public T getValue(JobId jobId, ITupleReference tuple)
- throws MetadataException, HyracksDataException, RemoteException;
+ T getValue(JobId jobId, ITupleReference tuple) throws AlgebricksException, HyracksDataException, RemoteException;
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index d2622c4..bff721e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -70,6 +70,7 @@
import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerFactory;
import org.apache.asterix.transaction.management.resource.DatasetLocalResourceFactory;
import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.INCServiceContext;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
@@ -127,11 +128,11 @@
* @param isNewUniverse
* @throws ACIDException
* @throws RemoteException
- * @throws MetadataException
+ * @throws AlgebricksException
* @throws Exception
*/
public static void startUniverse(INCServiceContext ncServiceContext, boolean isNewUniverse)
- throws RemoteException, ACIDException, MetadataException {
+ throws RemoteException, ACIDException, AlgebricksException {
MetadataBootstrap.setNewUniverse(isNewUniverse);
appContext = (INcApplicationContext) ncServiceContext.getApplicationContext();
@@ -185,7 +186,7 @@
}
}
- private static void insertInitialDataverses(MetadataTransactionContext mdTxnCtx) throws MetadataException {
+ private static void insertInitialDataverses(MetadataTransactionContext mdTxnCtx) throws AlgebricksException {
String dataFormat = NonTaggedDataFormat.NON_TAGGED_DATA_FORMAT;
MetadataManager.INSTANCE.addDataverse(mdTxnCtx,
new Dataverse(MetadataConstants.METADATA_DATAVERSE_NAME, dataFormat, MetadataUtil.PENDING_NO_OP));
@@ -193,15 +194,15 @@
}
/**
- * Inserts a metadata dataset to the physical dataset index
- * Should be performed on a bootstrap of a new universe
+ * Inserts a metadata dataset to the physical dataset index Should be performed
+ * on a bootstrap of a new universe
*
* @param mdTxnCtx
* @param indexes
- * @throws MetadataException
+ * @throws AlgebricksException
*/
public static void insertMetadataDatasets(MetadataTransactionContext mdTxnCtx, IMetadataIndex[] indexes)
- throws MetadataException {
+ throws AlgebricksException {
for (int i = 0; i < indexes.length; i++) {
IDatasetDetails id = new InternalDatasetDetails(FileStructure.BTREE, PartitioningStrategy.HASH,
indexes[i].getPartitioningExpr(), indexes[i].getPartitioningExpr(), null,
@@ -224,7 +225,7 @@
}
}
- private static void insertMetadataDatatypes(MetadataTransactionContext mdTxnCtx) throws MetadataException {
+ private static void insertMetadataDatatypes(MetadataTransactionContext mdTxnCtx) throws AlgebricksException {
ArrayList<IAType> types = new ArrayList<>();
types.addAll(BuiltinTypeMap.getAllBuiltinTypes());
getMetadataTypes(types);
@@ -238,20 +239,20 @@
}
}
- private static void insertNodes(MetadataTransactionContext mdTxnCtx) throws MetadataException {
+ private static void insertNodes(MetadataTransactionContext mdTxnCtx) throws AlgebricksException {
for (String nodesName : nodeNames) {
MetadataManager.INSTANCE.addNode(mdTxnCtx, new Node(nodesName, 0, 0));
}
}
- private static void insertInitialGroups(MetadataTransactionContext mdTxnCtx) throws MetadataException {
+ private static void insertInitialGroups(MetadataTransactionContext mdTxnCtx) throws AlgebricksException {
List<String> metadataGroupNodeNames = new ArrayList<>();
metadataGroupNodeNames.add(metadataNodeName);
NodeGroup groupRecord = new NodeGroup(MetadataConstants.METADATA_NODEGROUP_NAME, metadataGroupNodeNames);
MetadataManager.INSTANCE.addNodegroup(mdTxnCtx, groupRecord);
}
- private static void insertInitialAdapters(MetadataTransactionContext mdTxnCtx) throws MetadataException {
+ private static void insertInitialAdapters(MetadataTransactionContext mdTxnCtx) throws AlgebricksException {
String[] builtInAdapterClassNames = new String[] { GenericAdapterFactory.class.getName() };
DatasourceAdapter adapter;
for (String adapterClassName : builtInAdapterClassNames) {
@@ -263,7 +264,8 @@
}
}
- private static void insertInitialCompactionPolicies(MetadataTransactionContext mdTxnCtx) throws MetadataException {
+ private static void insertInitialCompactionPolicies(MetadataTransactionContext mdTxnCtx)
+ throws AlgebricksException {
String[] builtInCompactionPolicyClassNames =
new String[] { ConstantMergePolicyFactory.class.getName(), PrefixMergePolicyFactory.class.getName(),
NoMergePolicyFactory.class.getName(), CorrelatedPrefixMergePolicyFactory.class.getName() };
@@ -273,7 +275,7 @@
}
}
- private static DatasourceAdapter getAdapter(String adapterFactoryClassName) throws MetadataException {
+ private static DatasourceAdapter getAdapter(String adapterFactoryClassName) throws AlgebricksException {
try {
String adapterName = ((IAdapterFactory) (Class.forName(adapterFactoryClassName).newInstance())).getAlias();
return new DatasourceAdapter(new AdapterIdentifier(MetadataConstants.METADATA_DATAVERSE_NAME, adapterName),
@@ -284,7 +286,7 @@
}
private static CompactionPolicy getCompactionPolicyEntity(String compactionPolicyClassName)
- throws MetadataException {
+ throws AlgebricksException {
try {
String policyName =
((ILSMMergePolicyFactory) (Class.forName(compactionPolicyClassName).newInstance())).getName();
@@ -296,8 +298,8 @@
}
/**
- * Enlist a metadata index so it is available for metadata operations
- * should be performed upon bootstrapping
+ * Enlist a metadata index so it is available for metadata operations should be
+ * performed upon bootstrapping
*
* @param index
* @throws HyracksDataException
@@ -320,8 +322,10 @@
IBinaryComparatorFactory[] cmpFactories = index.getKeyBinaryComparatorFactory();
int[] bloomFilterKeyFields = index.getBloomFilterKeyFields();
- // opTrackerProvider and ioOpCallbackFactory should both be acquired through IStorageManager
- // We are unable to do this since IStorageManager needs a dataset to determine the appropriate
+ // opTrackerProvider and ioOpCallbackFactory should both be acquired through
+ // IStorageManager
+ // We are unable to do this since IStorageManager needs a dataset to determine
+ // the appropriate
// objects
ILSMOperationTrackerFactory opTrackerFactory =
index.isPrimaryIndex() ? new PrimaryIndexOperationTrackerFactory(datasetId)
@@ -341,7 +345,8 @@
bloomFilterKeyFields, appContext.getBloomFilterFalsePositiveRate(), true, null);
DatasetLocalResourceFactory dsLocalResourceFactory =
new DatasetLocalResourceFactory(datasetId, lsmBtreeFactory);
- // TODO(amoudi) Creating the index should be done through the same code path as other indexes
+ // TODO(amoudi) Creating the index should be done through the same code path as
+ // other indexes
// This is to be done by having a metadata dataset associated with each index
IIndexBuilder indexBuilder = new IndexBuilder(ncServiceCtx, storageComponentProvider.getStorageManager(),
index::getResourceId, file, dsLocalResourceFactory, true);
@@ -354,7 +359,8 @@
.get(appContext.getTransactionSubsystem().getId())
+ " to intialize as a new instance. (WARNING: all data will be lost.)");
}
- // Why do we care about metadata dataset's resource ids? why not assign them ids similar to other resources?
+ // Why do we care about metadata dataset's resource ids? why not assign them ids
+ // similar to other resources?
if (index.getResourceId() != resource.getId()) {
throw new HyracksDataException("Resource Id doesn't match expected metadata index resource id");
}
@@ -368,9 +374,10 @@
/**
* Perform recovery of DDL operations metadata records
*/
- public static void startDDLRecovery() throws MetadataException {
+ public static void startDDLRecovery() throws AlgebricksException {
// #. clean up any record which has pendingAdd/DelOp flag
- // as traversing all records from DATAVERSE_DATASET to DATASET_DATASET, and then to INDEX_DATASET.
+ // as traversing all records from DATAVERSE_DATASET to DATASET_DATASET, and then
+ // to INDEX_DATASET.
MetadataTransactionContext mdTxnCtx = null;
MetadataManager.INSTANCE.acquireWriteLatch();
if (LOGGER.isLoggable(Level.INFO)) {
@@ -383,7 +390,8 @@
for (Dataverse dataverse : dataverses) {
recoverDataverse(mdTxnCtx, dataverse);
}
- // the commit wasn't there before. yet, everything was working correctly!!!!!!!!!!!
+ // the commit wasn't there before. yet, everything was working
+ // correctly!!!!!!!!!!!
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Completed DDL recovery.");
@@ -404,7 +412,7 @@
}
private static void recoverDataverse(MetadataTransactionContext mdTxnCtx, Dataverse dataverse)
- throws MetadataException {
+ throws AlgebricksException {
if (dataverse.getPendingOp() != MetadataUtil.PENDING_NO_OP) {
// drop pending dataverse
MetadataManager.INSTANCE.dropDataverse(mdTxnCtx, dataverse.getDataverseName());
@@ -420,7 +428,8 @@
}
}
- private static void recoverDataset(MetadataTransactionContext mdTxnCtx, Dataset dataset) throws MetadataException {
+ private static void recoverDataset(MetadataTransactionContext mdTxnCtx, Dataset dataset)
+ throws AlgebricksException {
if (dataset.getPendingOp() != MetadataUtil.PENDING_NO_OP) {
// drop pending dataset
MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataset.getDataverseName(), dataset.getDatasetName());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
index 97c6ed2..abadfb2 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
@@ -23,7 +23,6 @@
import org.apache.asterix.common.cluster.IClusterStateManager;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.MetadataTransactionContext;
import org.apache.asterix.metadata.entities.Dataset;
@@ -52,13 +51,7 @@
if (dataverse == null || typeName == null) {
return null;
}
- Datatype type;
- try {
- type = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverse, typeName);
- } catch (MetadataException e) {
- throw new AlgebricksException(
- "Metadata exception while looking up type '" + typeName + "' in dataverse '" + dataverse + "'", e);
- }
+ Datatype type = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverse, typeName);
if (type == null) {
throw new AlgebricksException("Type name '" + typeName + "' unknown in dataverse '" + dataverse + "'");
}
@@ -81,7 +74,7 @@
}
public static DatasourceAdapter getAdapter(MetadataTransactionContext mdTxnCtx, String dataverseName,
- String adapterName) throws MetadataException {
+ String adapterName) throws AlgebricksException {
DatasourceAdapter adapter;
// search in default namespace (built-in adapter)
adapter = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME, adapterName);
@@ -95,11 +88,7 @@
public static Dataset findDataset(MetadataTransactionContext mdTxnCtx, String dataverse, String dataset)
throws AlgebricksException {
- try {
- return MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverse, dataset);
- } catch (MetadataException e) {
- throw new AlgebricksException(e);
- }
+ return MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverse, dataset);
}
public static Dataset findExistingDataset(MetadataTransactionContext mdTxnCtx, String dataverseName,
@@ -131,47 +120,27 @@
public static Feed findFeed(MetadataTransactionContext mdTxnCtx, String dataverse, String feedName)
throws AlgebricksException {
- try {
- return MetadataManager.INSTANCE.getFeed(mdTxnCtx, dataverse, feedName);
- } catch (MetadataException e) {
- throw new AlgebricksException(e);
- }
+ return MetadataManager.INSTANCE.getFeed(mdTxnCtx, dataverse, feedName);
}
public static FeedConnection findFeedConnection(MetadataTransactionContext mdTxnCtx, String dataverse,
String feedName, String datasetName) throws AlgebricksException {
- try {
- return MetadataManager.INSTANCE.getFeedConnection(mdTxnCtx, dataverse, feedName, datasetName);
- } catch (MetadataException e) {
- throw new AlgebricksException(e);
- }
+ return MetadataManager.INSTANCE.getFeedConnection(mdTxnCtx, dataverse, feedName, datasetName);
}
public static FeedPolicyEntity findFeedPolicy(MetadataTransactionContext mdTxnCtx, String dataverse,
String policyName) throws AlgebricksException {
- try {
- return MetadataManager.INSTANCE.getFeedPolicy(mdTxnCtx, dataverse, policyName);
- } catch (MetadataException e) {
- throw new AlgebricksException(e);
- }
+ return MetadataManager.INSTANCE.getFeedPolicy(mdTxnCtx, dataverse, policyName);
}
public static List<Index> getDatasetIndexes(MetadataTransactionContext mdTxnCtx, String dataverseName,
String datasetName) throws AlgebricksException {
- try {
- return MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
- } catch (MetadataException e) {
- throw new AlgebricksException(e);
- }
+ return MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
}
public static DataSource findDataSource(IClusterStateManager clusterStateManager,
MetadataTransactionContext mdTxnCtx, DataSourceId id) throws AlgebricksException {
- try {
- return lookupSourceInMetadata(clusterStateManager, mdTxnCtx, id);
- } catch (MetadataException e) {
- throw new AlgebricksException(e);
- }
+ return lookupSourceInMetadata(clusterStateManager, mdTxnCtx, id);
}
public static DataSource lookupSourceInMetadata(IClusterStateManager clusterStateManager,
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index fb9901d..463c96b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -36,8 +36,6 @@
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.common.dataflow.LSMTreeInsertDeleteOperatorDescriptor;
import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.common.metadata.LockList;
import org.apache.asterix.common.transactions.JobId;
import org.apache.asterix.common.utils.StoragePathUtil;
@@ -57,7 +55,6 @@
import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.external.util.FeedConstants;
import org.apache.asterix.formats.base.IDataFormat;
-import org.apache.asterix.om.functions.IFunctionExtensionManager;
import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
import org.apache.asterix.formats.nontagged.LinearizeComparatorFactoryProvider;
import org.apache.asterix.formats.nontagged.TypeTraitProvider;
@@ -79,6 +76,7 @@
import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.asterix.metadata.utils.SplitsAndConstraintsUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionExtensionManager;
import org.apache.asterix.om.functions.IFunctionManager;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
@@ -399,7 +397,7 @@
return new Pair<>(dataScanner, constraint);
}
- public Dataverse findDataverse(String dataverseName) throws CompilationException {
+ public Dataverse findDataverse(String dataverseName) throws AlgebricksException {
return MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
}
@@ -437,47 +435,39 @@
boolean lowKeyInclusive, boolean highKeyInclusive, boolean propagateFilter, int[] minFilterFieldIndexes,
int[] maxFilterFieldIndexes) throws AlgebricksException {
boolean isSecondary = true;
- try {
- Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
- dataset.getDatasetName(), dataset.getDatasetName());
- if (primaryIndex != null && (dataset.getDatasetType() != DatasetType.EXTERNAL)) {
- isSecondary = !indexName.equals(primaryIndex.getIndexName());
- }
- Index theIndex =
- isSecondary
- ? MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
- dataset.getDatasetName(), indexName)
- : primaryIndex;
- int numPrimaryKeys = dataset.getPrimaryKeys().size();
- RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc =
- getSplitProviderAndConstraints(dataset, theIndex.getIndexName());
- int[] primaryKeyFields = new int[numPrimaryKeys];
- for (int i = 0; i < numPrimaryKeys; i++) {
- primaryKeyFields[i] = i;
- }
-
- ISearchOperationCallbackFactory searchCallbackFactory = dataset.getSearchCallbackFactory(
- storageComponentProvider, theIndex, jobId, IndexOperation.SEARCH, primaryKeyFields);
- IStorageManager storageManager = getStorageComponentProvider().getStorageManager();
- IIndexDataflowHelperFactory indexHelperFactory = new IndexDataflowHelperFactory(storageManager, spPc.first);
- BTreeSearchOperatorDescriptor btreeSearchOp;
- if (dataset.getDatasetType() == DatasetType.INTERNAL) {
- btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields, highKeyFields,
- lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
- context.getMissingWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, propagateFilter);
- } else {
- btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
- highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput,
- retainMissing, context.getMissingWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes,
- ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this));
- }
- return new Pair<>(btreeSearchOp, spPc.second);
- } catch (MetadataException me) {
- throw new AlgebricksException(me);
+ Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+ dataset.getDatasetName(), dataset.getDatasetName());
+ if (primaryIndex != null && (dataset.getDatasetType() != DatasetType.EXTERNAL)) {
+ isSecondary = !indexName.equals(primaryIndex.getIndexName());
}
+ Index theIndex = isSecondary ? MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+ dataset.getDatasetName(), indexName) : primaryIndex;
+ int numPrimaryKeys = dataset.getPrimaryKeys().size();
+ RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc =
+ getSplitProviderAndConstraints(dataset, theIndex.getIndexName());
+ int[] primaryKeyFields = new int[numPrimaryKeys];
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ primaryKeyFields[i] = i;
+ }
+
+ ISearchOperationCallbackFactory searchCallbackFactory = dataset.getSearchCallbackFactory(
+ storageComponentProvider, theIndex, jobId, IndexOperation.SEARCH, primaryKeyFields);
+ IStorageManager storageManager = getStorageComponentProvider().getStorageManager();
+ IIndexDataflowHelperFactory indexHelperFactory = new IndexDataflowHelperFactory(storageManager, spPc.first);
+ BTreeSearchOperatorDescriptor btreeSearchOp;
+ if (dataset.getDatasetType() == DatasetType.INTERNAL) {
+ btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields, highKeyFields,
+ lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
+ context.getMissingWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
+ maxFilterFieldIndexes, propagateFilter);
+ } else {
+ btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
+ highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
+ context.getMissingWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
+ maxFilterFieldIndexes, ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this));
+ }
+ return new Pair<>(btreeSearchOp, spPc.second);
}
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildRtreeRuntime(JobSpecification jobSpec,
@@ -485,43 +475,39 @@
JobGenContext context, boolean retainInput, boolean retainMissing, Dataset dataset, String indexName,
int[] keyFields, boolean propagateFilter, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes)
throws AlgebricksException {
- try {
- int numPrimaryKeys = dataset.getPrimaryKeys().size();
- Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
- dataset.getDatasetName(), indexName);
- if (secondaryIndex == null) {
- throw new AlgebricksException(
- "Code generation error: no index " + indexName + " for dataset " + dataset.getDatasetName());
- }
- RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc =
- getSplitProviderAndConstraints(dataset, secondaryIndex.getIndexName());
- int[] primaryKeyFields = new int[numPrimaryKeys];
- for (int i = 0; i < numPrimaryKeys; i++) {
- primaryKeyFields[i] = i;
- }
-
- ISearchOperationCallbackFactory searchCallbackFactory = dataset.getSearchCallbackFactory(
- storageComponentProvider, secondaryIndex, jobId, IndexOperation.SEARCH, primaryKeyFields);
- RTreeSearchOperatorDescriptor rtreeSearchOp;
- IIndexDataflowHelperFactory indexDataflowHelperFactory =
- new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), spPc.first);
- if (dataset.getDatasetType() == DatasetType.INTERNAL) {
- rtreeSearchOp = new RTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, keyFields, true, true,
- indexDataflowHelperFactory, retainInput, retainMissing, context.getMissingWriterFactory(),
- searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes, propagateFilter);
- } else {
- // Create the operator
- rtreeSearchOp = new ExternalRTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, keyFields, true, true,
- indexDataflowHelperFactory, retainInput, retainMissing, context.getMissingWriterFactory(),
- searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes,
- ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this));
- }
-
- return new Pair<>(rtreeSearchOp, spPc.second);
- } catch (MetadataException me) {
- throw new AlgebricksException(me);
+ int numPrimaryKeys = dataset.getPrimaryKeys().size();
+ Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+ dataset.getDatasetName(), indexName);
+ if (secondaryIndex == null) {
+ throw new AlgebricksException(
+ "Code generation error: no index " + indexName + " for dataset " + dataset.getDatasetName());
}
+ RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc =
+ getSplitProviderAndConstraints(dataset, secondaryIndex.getIndexName());
+ int[] primaryKeyFields = new int[numPrimaryKeys];
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ primaryKeyFields[i] = i;
+ }
+
+ ISearchOperationCallbackFactory searchCallbackFactory = dataset.getSearchCallbackFactory(
+ storageComponentProvider, secondaryIndex, jobId, IndexOperation.SEARCH, primaryKeyFields);
+ RTreeSearchOperatorDescriptor rtreeSearchOp;
+ IIndexDataflowHelperFactory indexDataflowHelperFactory =
+ new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), spPc.first);
+ if (dataset.getDatasetType() == DatasetType.INTERNAL) {
+ rtreeSearchOp = new RTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, keyFields, true, true,
+ indexDataflowHelperFactory, retainInput, retainMissing, context.getMissingWriterFactory(),
+ searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes, propagateFilter);
+ } else {
+ // Create the operator
+ rtreeSearchOp = new ExternalRTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, keyFields, true, true,
+ indexDataflowHelperFactory, retainInput, retainMissing, context.getMissingWriterFactory(),
+ searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes,
+ ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this));
+ }
+
+ return new Pair<>(rtreeSearchOp, spPc.second);
}
@Override
@@ -583,26 +569,22 @@
fieldPermutation[numKeys + 1] = idx;
}
- try {
- boolean temp = dataset.getDatasetDetails().isTemp();
- isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob && temp;
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
- getSplitProviderAndConstraints(dataset);
- long numElementsHint = getCardinalityPerPartitionHint(dataset);
- // TODO
- // figure out the right behavior of the bulkload and then give the
- // right callback
- // (ex. what's the expected behavior when there is an error during
- // bulkload?)
- IIndexDataflowHelperFactory indexHelperFactory = new IndexDataflowHelperFactory(
- storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
- LSMIndexBulkLoadOperatorDescriptor btreeBulkLoad = new LSMIndexBulkLoadOperatorDescriptor(spec, null,
- fieldPermutation, GlobalConfig.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, true,
- indexHelperFactory, null, BulkLoadUsage.LOAD, dataset.getDatasetId());
- return new Pair<>(btreeBulkLoad, splitsAndConstraint.second);
- } catch (MetadataException me) {
- throw new AlgebricksException(me);
- }
+ boolean temp = dataset.getDatasetDetails().isTemp();
+ isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob && temp;
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+ getSplitProviderAndConstraints(dataset);
+ long numElementsHint = getCardinalityPerPartitionHint(dataset);
+ // TODO
+ // figure out the right behavior of the bulkload and then give the
+ // right callback
+ // (ex. what's the expected behavior when there is an error during
+ // bulkload?)
+ IIndexDataflowHelperFactory indexHelperFactory =
+ new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
+ LSMIndexBulkLoadOperatorDescriptor btreeBulkLoad = new LSMIndexBulkLoadOperatorDescriptor(spec, null,
+ fieldPermutation, GlobalConfig.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, true,
+ indexHelperFactory, null, BulkLoadUsage.LOAD, dataset.getDatasetId());
+ return new Pair<>(btreeBulkLoad, splitsAndConstraint.second);
}
@Override
@@ -680,13 +662,8 @@
}
Dataset dataset = MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataverseName, datasetName);
- Index secondaryIndex;
- try {
- secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
- dataset.getDatasetName(), indexName);
- } catch (MetadataException e) {
- throw new AlgebricksException(e);
- }
+ Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+ dataset.getDatasetName(), indexName);
// TokenizeOperator only supports a keyword or n-gram index.
switch (secondaryIndex.getIndexType()) {
case SINGLE_PARTITION_WORD_INVIX:
@@ -703,8 +680,8 @@
/**
* Calculate an estimate size of the bloom filter. Note that this is an
- * estimation which assumes that the data is going to be uniformly
- * distributed across all partitions.
+ * estimation which assumes that the data is going to be uniformly distributed
+ * across all partitions.
*
* @param dataset
* @return Number of elements that will be used to create a bloom filter per
@@ -779,7 +756,7 @@
}
public DatasourceAdapter getAdapter(MetadataTransactionContext mdTxnCtx, String dataverseName, String adapterName)
- throws MetadataException {
+ throws AlgebricksException {
DatasourceAdapter adapter;
// search in default namespace (built-in adapter)
adapter = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME, adapterName);
@@ -816,7 +793,8 @@
// Create the file index data flow helper
IIndexDataflowHelperFactory indexDataflowHelperFactory =
new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), spPc.first);
- // Create the out record descriptor, appContext and fileSplitProvider for the files index
+ // Create the out record descriptor, appContext and fileSplitProvider for the
+ // files index
RecordDescriptor outRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
ISearchOperationCallbackFactory searchOpCallbackFactory = dataset
.getSearchCallbackFactory(storageComponentProvider, fileIndex, jobId, IndexOperation.SEARCH, null);
@@ -877,8 +855,7 @@
}
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDatasetDataScannerRuntime(
- JobSpecification jobSpec, IAType itemType, IAdapterFactory adapterFactory)
- throws AlgebricksException {
+ JobSpecification jobSpec, IAType itemType, IAdapterFactory adapterFactory) throws AlgebricksException {
if (itemType.getTypeTag() != ATypeTag.OBJECT) {
throw new AlgebricksException("Can only scan datasets of records.");
}
@@ -984,35 +961,31 @@
}
}
- try {
- Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
- dataset.getDatasetName(), dataset.getDatasetName());
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
- getSplitProviderAndConstraints(dataset);
+ Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+ dataset.getDatasetName(), dataset.getDatasetName());
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+ getSplitProviderAndConstraints(dataset);
- // prepare callback
- int[] primaryKeyFields = new int[numKeys];
- for (i = 0; i < numKeys; i++) {
- primaryKeyFields[i] = i;
- }
- IModificationOperationCallbackFactory modificationCallbackFactory = dataset.getModificationCallbackFactory(
- storageComponentProvider, primaryIndex, jobId, indexOp, primaryKeyFields);
- IIndexDataflowHelperFactory idfh = new IndexDataflowHelperFactory(
- storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
- IOperatorDescriptor op;
- if (bulkload) {
- long numElementsHint = getCardinalityPerPartitionHint(dataset);
- op = new LSMIndexBulkLoadOperatorDescriptor(spec, inputRecordDesc, fieldPermutation,
- GlobalConfig.DEFAULT_TREE_FILL_FACTOR, true, numElementsHint, true, idfh, null,
- BulkLoadUsage.LOAD, dataset.getDatasetId());
- } else {
- op = new LSMTreeInsertDeleteOperatorDescriptor(spec, inputRecordDesc, fieldPermutation, indexOp, idfh,
- null, true, modificationCallbackFactory);
- }
- return new Pair<>(op, splitsAndConstraint.second);
- } catch (MetadataException me) {
- throw new AlgebricksException(me);
+ // prepare callback
+ int[] primaryKeyFields = new int[numKeys];
+ for (i = 0; i < numKeys; i++) {
+ primaryKeyFields[i] = i;
}
+ IModificationOperationCallbackFactory modificationCallbackFactory = dataset.getModificationCallbackFactory(
+ storageComponentProvider, primaryIndex, jobId, indexOp, primaryKeyFields);
+ IIndexDataflowHelperFactory idfh =
+ new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
+ IOperatorDescriptor op;
+ if (bulkload) {
+ long numElementsHint = getCardinalityPerPartitionHint(dataset);
+ op = new LSMIndexBulkLoadOperatorDescriptor(spec, inputRecordDesc, fieldPermutation,
+ GlobalConfig.DEFAULT_TREE_FILL_FACTOR, true, numElementsHint, true, idfh, null, BulkLoadUsage.LOAD,
+ dataset.getDatasetId());
+ } else {
+ op = new LSMTreeInsertDeleteOperatorDescriptor(spec, inputRecordDesc, fieldPermutation, indexOp, idfh, null,
+ true, modificationCallbackFactory);
+ }
+ return new Pair<>(op, splitsAndConstraint.second);
}
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertOrDeleteOrUpsertRuntime(
@@ -1028,13 +1001,8 @@
String datasetName = dataSourceIndex.getDataSource().getId().getDatasourceName();
Dataset dataset = MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataverseName, datasetName);
- Index secondaryIndex;
- try {
- secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
- dataset.getDatasetName(), indexName);
- } catch (MetadataException e) {
- throw new AlgebricksException(e);
- }
+ Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+ dataset.getDatasetName(), indexName);
ArrayList<LogicalVariable> prevAdditionalFilteringKeys = null;
if (indexOp == IndexOperation.UPSERT && prevAdditionalFilteringKey != null) {
@@ -1159,101 +1127,96 @@
AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc, JobGenContext context,
JobSpecification spec, IndexOperation indexOp, boolean bulkload, List<LogicalVariable> prevSecondaryKeys,
List<LogicalVariable> prevAdditionalFilteringKeys) throws AlgebricksException {
+ Dataset dataset = MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataverseName, datasetName);
+ boolean temp = dataset.getDatasetDetails().isTemp();
+ isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob && temp;
- try {
- Dataset dataset = MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataverseName, datasetName);
- boolean temp = dataset.getDatasetDetails().isTemp();
- isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob && temp;
+ String itemTypeName = dataset.getItemTypeName();
+ IAType itemType = MetadataManager.INSTANCE
+ .getDatatype(mdTxnCtx, dataset.getItemTypeDataverseName(), itemTypeName).getDatatype();
+ validateRecordType(itemType);
+ ARecordType recType = (ARecordType) itemType;
+ Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+ dataset.getDatasetName(), indexName);
+ List<List<String>> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
+ List<IAType> secondaryKeyTypes = secondaryIndex.getKeyFieldTypes();
+ Pair<IAType, Boolean> keyPairType =
+ Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyExprs.get(0), recType);
+ IAType spatialType = keyPairType.first;
+ int dimension = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
+ int numSecondaryKeys = dimension * 2;
+ int numPrimaryKeys = primaryKeys.size();
+ int numKeys = numSecondaryKeys + numPrimaryKeys;
- String itemTypeName = dataset.getItemTypeName();
- IAType itemType = MetadataManager.INSTANCE
- .getDatatype(mdTxnCtx, dataset.getItemTypeDataverseName(), itemTypeName).getDatatype();
- validateRecordType(itemType);
- ARecordType recType = (ARecordType) itemType;
- Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
- dataset.getDatasetName(), indexName);
- List<List<String>> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
- List<IAType> secondaryKeyTypes = secondaryIndex.getKeyFieldTypes();
- Pair<IAType, Boolean> keyPairType =
- Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyExprs.get(0), recType);
- IAType spatialType = keyPairType.first;
- int dimension = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
- int numSecondaryKeys = dimension * 2;
- int numPrimaryKeys = primaryKeys.size();
- int numKeys = numSecondaryKeys + numPrimaryKeys;
+ int numFilterFields = DatasetUtil.getFilterField(dataset) == null ? 0 : 1;
+ int[] fieldPermutation = new int[numKeys + numFilterFields];
+ int[] modificationCallbackPrimaryKeyFields = new int[primaryKeys.size()];
+ int i = 0;
+ int j = 0;
- int numFilterFields = DatasetUtil.getFilterField(dataset) == null ? 0 : 1;
- int[] fieldPermutation = new int[numKeys + numFilterFields];
- int[] modificationCallbackPrimaryKeyFields = new int[primaryKeys.size()];
- int i = 0;
- int j = 0;
+ for (LogicalVariable varKey : secondaryKeys) {
+ int idx = propagatedSchema.findVariable(varKey);
+ fieldPermutation[i] = idx;
+ i++;
+ }
+ for (LogicalVariable varKey : primaryKeys) {
+ int idx = propagatedSchema.findVariable(varKey);
+ fieldPermutation[i] = idx;
+ modificationCallbackPrimaryKeyFields[j] = i;
+ i++;
+ j++;
+ }
- for (LogicalVariable varKey : secondaryKeys) {
+ if (numFilterFields > 0) {
+ int idx = propagatedSchema.findVariable(additionalNonKeyFields.get(0));
+ fieldPermutation[numKeys] = idx;
+ }
+
+ int[] prevFieldPermutation = null;
+ if (indexOp == IndexOperation.UPSERT) {
+ // Get field permutation for previous value
+ prevFieldPermutation = new int[numKeys + numFilterFields];
+ i = 0;
+
+ // Get field permutation for new value
+ for (LogicalVariable varKey : prevSecondaryKeys) {
int idx = propagatedSchema.findVariable(varKey);
- fieldPermutation[i] = idx;
+ prevFieldPermutation[i] = idx;
i++;
}
- for (LogicalVariable varKey : primaryKeys) {
- int idx = propagatedSchema.findVariable(varKey);
- fieldPermutation[i] = idx;
- modificationCallbackPrimaryKeyFields[j] = i;
+ for (int k = 0; k < numPrimaryKeys; k++) {
+ prevFieldPermutation[k + i] = fieldPermutation[k + i];
i++;
- j++;
}
if (numFilterFields > 0) {
- int idx = propagatedSchema.findVariable(additionalNonKeyFields.get(0));
- fieldPermutation[numKeys] = idx;
+ int idx = propagatedSchema.findVariable(prevAdditionalFilteringKeys.get(0));
+ prevFieldPermutation[numKeys] = idx;
}
-
- int[] prevFieldPermutation = null;
- if (indexOp == IndexOperation.UPSERT) {
- // Get field permutation for previous value
- prevFieldPermutation = new int[numKeys + numFilterFields];
- i = 0;
-
- // Get field permutation for new value
- for (LogicalVariable varKey : prevSecondaryKeys) {
- int idx = propagatedSchema.findVariable(varKey);
- prevFieldPermutation[i] = idx;
- i++;
- }
- for (int k = 0; k < numPrimaryKeys; k++) {
- prevFieldPermutation[k + i] = fieldPermutation[k + i];
- i++;
- }
-
- if (numFilterFields > 0) {
- int idx = propagatedSchema.findVariable(prevAdditionalFilteringKeys.get(0));
- prevFieldPermutation[numKeys] = idx;
- }
- }
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
- getSplitProviderAndConstraints(dataset, secondaryIndex.getIndexName());
-
- // prepare callback
- JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
- IModificationOperationCallbackFactory modificationCallbackFactory = dataset.getModificationCallbackFactory(
- storageComponentProvider, secondaryIndex, jobId, indexOp, modificationCallbackPrimaryKeyFields);
- IIndexDataflowHelperFactory indexDataflowHelperFactory = new IndexDataflowHelperFactory(
- storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
- IOperatorDescriptor op;
- if (bulkload) {
- long numElementsHint = getCardinalityPerPartitionHint(dataset);
- op = new LSMIndexBulkLoadOperatorDescriptor(spec, recordDesc, fieldPermutation,
- GlobalConfig.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, false,
- indexDataflowHelperFactory, null, BulkLoadUsage.LOAD, dataset.getDatasetId());
- } else if (indexOp == IndexOperation.UPSERT) {
- op = new LSMSecondaryUpsertOperatorDescriptor(spec, recordDesc, fieldPermutation,
- indexDataflowHelperFactory, filterFactory, modificationCallbackFactory, prevFieldPermutation);
- } else {
- op = new LSMTreeInsertDeleteOperatorDescriptor(spec, recordDesc, fieldPermutation, indexOp,
- indexDataflowHelperFactory, filterFactory, false, modificationCallbackFactory);
- }
- return new Pair<>(op, splitsAndConstraint.second);
- } catch (MetadataException e) {
- throw new AlgebricksException(e);
}
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+ getSplitProviderAndConstraints(dataset, secondaryIndex.getIndexName());
+
+ // prepare callback
+ JobId planJobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
+ IModificationOperationCallbackFactory modificationCallbackFactory = dataset.getModificationCallbackFactory(
+ storageComponentProvider, secondaryIndex, planJobId, indexOp, modificationCallbackPrimaryKeyFields);
+ IIndexDataflowHelperFactory indexDataflowHelperFactory =
+ new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
+ IOperatorDescriptor op;
+ if (bulkload) {
+ long numElementsHint = getCardinalityPerPartitionHint(dataset);
+ op = new LSMIndexBulkLoadOperatorDescriptor(spec, recordDesc, fieldPermutation,
+ GlobalConfig.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, false, indexDataflowHelperFactory,
+ null, BulkLoadUsage.LOAD, dataset.getDatasetId());
+ } else if (indexOp == IndexOperation.UPSERT) {
+ op = new LSMSecondaryUpsertOperatorDescriptor(spec, recordDesc, fieldPermutation,
+ indexDataflowHelperFactory, filterFactory, modificationCallbackFactory, prevFieldPermutation);
+ } else {
+ op = new LSMTreeInsertDeleteOperatorDescriptor(spec, recordDesc, fieldPermutation, indexOp,
+ indexDataflowHelperFactory, filterFactory, false, modificationCallbackFactory);
+ }
+ return new Pair<>(op, splitsAndConstraint.second);
}
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInvertedIndexRuntime(String dataverseName,
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
index 1f0a08e..22732d3 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
@@ -31,6 +31,7 @@
import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
/**
* Maps from a string representation of an Asterix type to an Asterix type.
@@ -93,7 +94,7 @@
}
public static IAType getTypeFromTypeName(MetadataNode metadataNode, JobId jobId, String dataverseName,
- String typeName, boolean optional) throws MetadataException {
+ String typeName, boolean optional) throws AlgebricksException {
IAType type = _builtinTypeMap.get(typeName);
if (type == null) {
try {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/CompactionPolicyTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/CompactionPolicyTupleTranslator.java
index 3c409da..6595d97 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/CompactionPolicyTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/CompactionPolicyTupleTranslator.java
@@ -23,13 +23,13 @@
import java.io.DataInput;
import java.io.DataInputStream;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
import org.apache.asterix.metadata.entities.CompactionPolicy;
import org.apache.asterix.om.base.ARecord;
import org.apache.asterix.om.base.AString;
+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.dataflow.common.data.accessors.ITupleReference;
@@ -82,7 +82,7 @@
@Override
public ITupleReference getTupleFromMetadataEntity(CompactionPolicy compactionPolicy)
- throws HyracksDataException, MetadataException {
+ throws HyracksDataException, AlgebricksException {
tupleBuilder.reset();
aString.setValue(compactionPolicy.getDataverseName());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index 8da34c0..c274833 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -37,7 +37,6 @@
import org.apache.asterix.builders.UnorderedListBuilder;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.config.DatasetConfig.TransactionState;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.metadata.IDatasetDetails;
import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
@@ -65,6 +64,7 @@
import org.apache.asterix.om.types.AUnorderedListType;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
+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.util.ArrayBackedValueStorage;
@@ -207,8 +207,10 @@
}
}
- // Temporary dataset only lives in the compiler therefore the temp field is false.
- // DatasetTupleTranslator always read from the metadata node, so the temp flag should be always false.
+ // Temporary dataset only lives in the compiler therefore the temp field is
+ // false.
+ // DatasetTupleTranslator always read from the metadata node, so the temp flag
+ // should be always false.
datasetDetails = new InternalDatasetDetails(fileStructure, partitioningStrategy, partitioningKey,
partitioningKey, keyFieldSourceIndicator, partitioningKeyType, autogenerated, filterField,
false);
@@ -270,7 +272,8 @@
}
@Override
- public ITupleReference getTupleFromMetadataEntity(Dataset dataset) throws HyracksDataException, MetadataException {
+ public ITupleReference getTupleFromMetadataEntity(Dataset dataset)
+ throws HyracksDataException, AlgebricksException {
OrderedListBuilder listBuilder = new OrderedListBuilder();
ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
// write the key in the first 2 fields of the tuple
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
index 9cd73d7..a58e5d6 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
@@ -24,7 +24,6 @@
import java.io.DataInputStream;
import java.util.Calendar;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.external.api.IDataSourceAdapter;
import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
@@ -33,6 +32,7 @@
import org.apache.asterix.metadata.entities.DatasourceAdapter;
import org.apache.asterix.om.base.ARecord;
import org.apache.asterix.om.base.AString;
+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.dataflow.common.data.accessors.ITupleReference;
@@ -58,7 +58,7 @@
@Override
public DatasourceAdapter getMetadataEntityFromTuple(ITupleReference tuple)
- throws MetadataException, HyracksDataException {
+ throws AlgebricksException, HyracksDataException {
byte[] serRecord = tuple.getFieldData(ADAPTER_PAYLOAD_TUPLE_FIELD_INDEX);
int recordStartOffset = tuple.getFieldStart(ADAPTER_PAYLOAD_TUPLE_FIELD_INDEX);
int recordLength = tuple.getFieldLength(ADAPTER_PAYLOAD_TUPLE_FIELD_INDEX);
@@ -86,7 +86,7 @@
@Override
public ITupleReference getTupleFromMetadataEntity(DatasourceAdapter adapter)
- throws HyracksDataException, MetadataException {
+ throws HyracksDataException, AlgebricksException {
// write the key in the first 2 fields of the tuple
tupleBuilder.reset();
aString.setValue(adapter.getAdapterIdentifier().getNamespace());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
index fe0b40d..f75ea8f 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
@@ -29,7 +29,6 @@
import org.apache.asterix.builders.IARecordBuilder;
import org.apache.asterix.builders.OrderedListBuilder;
import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.common.transactions.JobId;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.metadata.MetadataNode;
@@ -51,6 +50,7 @@
import org.apache.asterix.om.types.AbstractComplexType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.ErrorCode;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -89,7 +89,7 @@
@Override
public Datatype getMetadataEntityFromTuple(ITupleReference frameTuple)
- throws MetadataException, HyracksDataException {
+ throws AlgebricksException, HyracksDataException {
byte[] serRecord = frameTuple.getFieldData(DATATYPE_PAYLOAD_TUPLE_FIELD_INDEX);
int recordStartOffset = frameTuple.getFieldStart(DATATYPE_PAYLOAD_TUPLE_FIELD_INDEX);
int recordLength = frameTuple.getFieldLength(DATATYPE_PAYLOAD_TUPLE_FIELD_INDEX);
@@ -99,7 +99,7 @@
return createDataTypeFromARecord(datatypeRecord);
}
- private Datatype createDataTypeFromARecord(ARecord datatypeRecord) throws MetadataException {
+ private Datatype createDataTypeFromARecord(ARecord datatypeRecord) throws AlgebricksException {
String dataverseName =
((AString) datatypeRecord.getValueByPos(MetadataRecordTypes.DATATYPE_ARECORD_DATAVERSENAME_FIELD_INDEX))
.getStringValue();
@@ -176,7 +176,7 @@
@Override
public ITupleReference getTupleFromMetadataEntity(Datatype dataType)
- throws HyracksDataException, MetadataException {
+ throws HyracksDataException, AlgebricksException {
// write the key in the first two fields of the tuple
tupleBuilder.reset();
aString.setValue(dataType.getDataverseName());
@@ -202,7 +202,7 @@
recordBuilder.addField(MetadataRecordTypes.DATATYPE_ARECORD_DATATYPENAME_FIELD_INDEX, fieldValue);
IAType fieldType = dataType.getDatatype();
- //unwrap nullable type out of the union
+ // unwrap nullable type out of the union
if (fieldType.getTypeTag() == ATypeTag.UNION) {
fieldType = ((AUnionType) dataType.getDatatype()).getActualType();
}
@@ -366,8 +366,9 @@
String dataverseName, String datatypeName) throws HyracksDataException {
try {
metadataNode.addDatatype(jobId, new Datatype(dataverseName, typeName, nestedType, true));
- } catch (MetadataException e) {
- // The nested record type may have been inserted by a previous DDL statement or by
+ } catch (AlgebricksException e) {
+ // The nested record type may have been inserted by a previous DDL statement or
+ // by
// a previous nested type.
if (!(e.getCause() instanceof HyracksDataException)) {
throw HyracksDataException.create(e);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
index 4b20a5b..3100b10 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
@@ -24,7 +24,6 @@
import java.io.DataInputStream;
import java.util.Calendar;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
@@ -34,6 +33,7 @@
import org.apache.asterix.om.base.ARecord;
import org.apache.asterix.om.base.AString;
import org.apache.asterix.om.types.BuiltinType;
+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.dataflow.common.data.accessors.ITupleReference;
@@ -77,7 +77,7 @@
@Override
public ITupleReference getTupleFromMetadataEntity(Dataverse instance)
- throws HyracksDataException, MetadataException {
+ throws HyracksDataException, AlgebricksException {
// write the key in the first field of the tuple
tupleBuilder.reset();
aString.setValue(instance.getDataverseName());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java
index a29048e..d058d90 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java
@@ -24,7 +24,6 @@
import java.util.Date;
import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.external.indexing.ExternalFile;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
@@ -38,6 +37,7 @@
import org.apache.asterix.om.base.ARecord;
import org.apache.asterix.om.base.AString;
import org.apache.asterix.om.types.BuiltinType;
+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.dataflow.common.data.accessors.ITupleReference;
@@ -76,7 +76,7 @@
@Override
public ExternalFile getMetadataEntityFromTuple(ITupleReference tuple)
- throws MetadataException, HyracksDataException {
+ throws AlgebricksException, HyracksDataException {
byte[] serRecord = tuple.getFieldData(EXTERNAL_FILE_PAYLOAD_TUPLE_FIELD_INDEX);
int recordStartOffset = tuple.getFieldStart(EXTERNAL_FILE_PAYLOAD_TUPLE_FIELD_INDEX);
int recordLength = tuple.getFieldLength(EXTERNAL_FILE_PAYLOAD_TUPLE_FIELD_INDEX);
@@ -108,7 +108,7 @@
@Override
public ITupleReference getTupleFromMetadataEntity(ExternalFile externalFile)
- throws MetadataException, HyracksDataException {
+ throws AlgebricksException, HyracksDataException {
// write the key in the first 3 fields of the tuple
tupleBuilder.reset();
// dataverse name
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedConnectionTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedConnectionTupleTranslator.java
index ac78b8f..269497b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedConnectionTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedConnectionTupleTranslator.java
@@ -27,7 +27,6 @@
import org.apache.asterix.builders.IARecordBuilder;
import org.apache.asterix.builders.UnorderedListBuilder;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
@@ -40,6 +39,7 @@
import org.apache.asterix.om.base.AUnorderedList;
import org.apache.asterix.om.base.IACursor;
import org.apache.asterix.om.types.AUnorderedListType;
+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.util.ArrayBackedValueStorage;
@@ -62,7 +62,7 @@
@Override
public FeedConnection getMetadataEntityFromTuple(ITupleReference frameTuple)
- throws MetadataException, HyracksDataException {
+ throws AlgebricksException, HyracksDataException {
byte[] serRecord = frameTuple.getFieldData(FEED_CONN_PAYLOAD_TUPLE_FIELD_INDEX);
int recordStartOffset = frameTuple.getFieldStart(FEED_CONN_PAYLOAD_TUPLE_FIELD_INDEX);
int recordLength = frameTuple.getFieldLength(FEED_CONN_PAYLOAD_TUPLE_FIELD_INDEX);
@@ -106,7 +106,7 @@
@Override
public ITupleReference getTupleFromMetadataEntity(FeedConnection me)
- throws MetadataException, HyracksDataException {
+ throws AlgebricksException, HyracksDataException {
tupleBuilder.reset();
// key: dataverse
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java
index ec22735..e80b96f4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java
@@ -29,7 +29,6 @@
import org.apache.asterix.builders.IARecordBuilder;
import org.apache.asterix.builders.RecordBuilder;
import org.apache.asterix.builders.UnorderedListBuilder;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
@@ -42,6 +41,7 @@
import org.apache.asterix.om.base.IACursor;
import org.apache.asterix.om.types.AUnorderedListType;
import org.apache.asterix.om.types.BuiltinType;
+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.util.ArrayBackedValueStorage;
@@ -110,7 +110,7 @@
@Override
public ITupleReference getTupleFromMetadataEntity(FeedPolicyEntity feedPolicy)
- throws HyracksDataException, MetadataException {
+ throws HyracksDataException, AlgebricksException {
// write the key in the first three fields of the tuple
ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java
index 2541e97..264da60 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java
@@ -30,7 +30,6 @@
import org.apache.asterix.builders.IARecordBuilder;
import org.apache.asterix.builders.RecordBuilder;
import org.apache.asterix.builders.UnorderedListBuilder;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
@@ -42,6 +41,7 @@
import org.apache.asterix.om.base.IACursor;
import org.apache.asterix.om.types.AUnorderedListType;
import org.apache.asterix.om.types.BuiltinType;
+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.util.ArrayBackedValueStorage;
@@ -109,7 +109,7 @@
}
@Override
- public ITupleReference getTupleFromMetadataEntity(Feed feed) throws HyracksDataException, MetadataException {
+ public ITupleReference getTupleFromMetadataEntity(Feed feed) throws HyracksDataException, AlgebricksException {
// write the key in the first two fields of the tuple
tupleBuilder.reset();
aString.setValue(feed.getDataverseName());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
index 858d443..e15805e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
@@ -26,7 +26,6 @@
import java.util.List;
import org.apache.asterix.builders.OrderedListBuilder;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
@@ -36,6 +35,7 @@
import org.apache.asterix.om.base.AString;
import org.apache.asterix.om.base.IACursor;
import org.apache.asterix.om.types.AOrderedListType;
+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.util.ArrayBackedValueStorage;
@@ -111,7 +111,7 @@
@Override
public ITupleReference getTupleFromMetadataEntity(Function function)
- throws HyracksDataException, MetadataException {
+ throws HyracksDataException, AlgebricksException {
// write the key in the first 2 fields of the tuple
tupleBuilder.reset();
aString.setValue(function.getDataverseName());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
index fec4ddc..8989cdc 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
@@ -29,7 +29,6 @@
import org.apache.asterix.builders.OrderedListBuilder;
import org.apache.asterix.common.config.DatasetConfig.IndexType;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.common.transactions.JobId;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.metadata.MetadataNode;
@@ -104,7 +103,8 @@
}
@Override
- public Index getMetadataEntityFromTuple(ITupleReference frameTuple) throws MetadataException, HyracksDataException {
+ public Index getMetadataEntityFromTuple(ITupleReference frameTuple)
+ throws AlgebricksException, HyracksDataException {
byte[] serRecord = frameTuple.getFieldData(INDEX_PAYLOAD_TUPLE_FIELD_INDEX);
int recordStartOffset = frameTuple.getFieldStart(INDEX_PAYLOAD_TUPLE_FIELD_INDEX);
int recordLength = frameTuple.getFieldLength(INDEX_PAYLOAD_TUPLE_FIELD_INDEX);
@@ -176,7 +176,8 @@
}
}
- // index key type information is not persisted, thus we extract type information from the record metadata
+ // index key type information is not persisted, thus we extract type information
+ // from the record metadata
if (searchKeyType.isEmpty()) {
try {
Dataset dSet = metadataNode.getDataset(jobId, dvName, dsName);
@@ -191,11 +192,7 @@
metaDt = (ARecordType) metadataNode.getDatatype(jobId, metatypeDataverseName, metatypeName)
.getDatatype();
}
- try {
- searchKeyType = KeyFieldTypeUtil.getKeyTypes(recordDt, metaDt, searchKey, keyFieldSourceIndicator);
- } catch (AlgebricksException e) {
- throw new MetadataException(e);
- }
+ searchKeyType = KeyFieldTypeUtil.getKeyTypes(recordDt, metaDt, searchKey, keyFieldSourceIndicator);
} catch (RemoteException re) {
throw HyracksDataException.create(re);
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
index 9c2b824..f001ec4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
@@ -24,13 +24,13 @@
import java.io.DataInputStream;
import java.util.Calendar;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
import org.apache.asterix.metadata.entities.Library;
import org.apache.asterix.om.base.ARecord;
import org.apache.asterix.om.base.AString;
+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.dataflow.common.data.accessors.ITupleReference;
@@ -79,7 +79,8 @@
}
@Override
- public ITupleReference getTupleFromMetadataEntity(Library library) throws HyracksDataException, MetadataException {
+ public ITupleReference getTupleFromMetadataEntity(Library library)
+ throws HyracksDataException, AlgebricksException {
// write the key in the first 2 fields of the tuple
tupleBuilder.reset();
aString.setValue(library.getDataverseName());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeGroupTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeGroupTupleTranslator.java
index 5329df4..5733de9 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeGroupTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeGroupTupleTranslator.java
@@ -27,7 +27,6 @@
import java.util.List;
import org.apache.asterix.builders.UnorderedListBuilder;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
@@ -37,6 +36,7 @@
import org.apache.asterix.om.base.AUnorderedList;
import org.apache.asterix.om.base.IACursor;
import org.apache.asterix.om.types.AUnorderedListType;
+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.util.ArrayBackedValueStorage;
@@ -86,7 +86,7 @@
@Override
public ITupleReference getTupleFromMetadataEntity(NodeGroup instance)
- throws HyracksDataException, MetadataException {
+ throws HyracksDataException, AlgebricksException {
// write the key in the first field of the tuple
tupleBuilder.reset();
aString.setValue(instance.getNodeGroupName());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeTupleTranslator.java
index b1847fd..f67ae3b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeTupleTranslator.java
@@ -27,6 +27,7 @@
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -84,7 +85,7 @@
}
@Override
- public ITupleReference getTupleFromMetadataEntity(Node instance) throws HyracksDataException, MetadataException {
+ public ITupleReference getTupleFromMetadataEntity(Node instance) throws HyracksDataException, AlgebricksException {
// write the key in the first field of the tuple
tupleBuilder.reset();
aString.setValue(instance.getNodeName());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/BuiltinFeedPolicies.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/BuiltinFeedPolicies.java
index 41b356a..5c67831 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/BuiltinFeedPolicies.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/BuiltinFeedPolicies.java
@@ -29,6 +29,7 @@
import org.apache.asterix.metadata.MetadataTransactionContext;
import org.apache.asterix.metadata.entities.FeedPolicyEntity;
import org.apache.asterix.metadata.utils.MetadataConstants;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
public class BuiltinFeedPolicies {
@@ -61,7 +62,7 @@
return null;
}
- //Basic
+ // Basic
private static FeedPolicyEntity initializeBasicPolicy() {
Map<String, String> policyParams = new HashMap<>();
policyParams.put(FeedPolicyAccessor.ELASTIC, Boolean.toString(false));
@@ -106,7 +107,7 @@
policyParams);
}
- public static void insertInitialFeedPolicies(MetadataTransactionContext mdTxnCtx) throws MetadataException {
+ public static void insertInitialFeedPolicies(MetadataTransactionContext mdTxnCtx) throws AlgebricksException {
for (FeedPolicyEntity feedPolicy : BuiltinFeedPolicies.POLICIES) {
MetadataManager.INSTANCE.addFeedPolicy(mdTxnCtx, feedPolicy);
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
index 3de83eb..c2d31bf 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
@@ -54,8 +54,8 @@
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
/**
- * A utility class for providing helper functions for feeds
- * TODO: Refactor this class.
+ * A utility class for providing helper functions for feeds TODO: Refactor this
+ * class.
*/
public class FeedMetadataUtil {
@@ -77,7 +77,7 @@
}
public static Feed validateIfFeedExists(String dataverse, String feedName, MetadataTransactionContext ctx)
- throws CompilationException {
+ throws AlgebricksException {
Feed feed = MetadataManager.INSTANCE.getFeed(ctx, dataverse, feedName);
if (feed == null) {
throw new CompilationException("Unknown source feed: " + feedName);
@@ -86,7 +86,7 @@
}
public static FeedPolicyEntity validateIfPolicyExists(String dataverse, String policyName,
- MetadataTransactionContext ctx) throws CompilationException {
+ MetadataTransactionContext ctx) throws AlgebricksException {
FeedPolicyEntity feedPolicy = MetadataManager.INSTANCE.getFeedPolicy(ctx, dataverse, policyName);
if (feedPolicy == null) {
feedPolicy =
@@ -273,7 +273,7 @@
}
public static ARecordType getOutputType(IFeed feed, Map<String, String> configuration, String key)
- throws MetadataException {
+ throws AlgebricksException {
ARecordType outputType = null;
String fqOutputType = configuration.get(key);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
index c0d9bda..90c0de3 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
@@ -52,7 +52,7 @@
private static Pattern unorderedListPattern = Pattern.compile("[{{*}}]");
public static IFunctionInfo getExternalFunctionInfo(MetadataTransactionContext txnCtx, Function function)
- throws MetadataException {
+ throws AlgebricksException {
String functionKind = function.getKind();
IFunctionInfo finfo = null;
@@ -69,9 +69,9 @@
}
private static IFunctionInfo getScalarFunctionInfo(MetadataTransactionContext txnCtx, Function function)
- throws MetadataException {
- FunctionIdentifier fid = new FunctionIdentifier(function.getDataverseName(), function.getName(),
- function.getArity());
+ throws AlgebricksException {
+ FunctionIdentifier fid =
+ new FunctionIdentifier(function.getDataverseName(), function.getName(), function.getArity());
IResultTypeComputer typeComputer = getResultTypeComputer(txnCtx, function);
List<IAType> arguments = new ArrayList<IAType>();
IAType returnType = null;
@@ -87,7 +87,7 @@
}
private static IAType getTypeInfo(String paramType, MetadataTransactionContext txnCtx, Function function)
- throws MetadataException {
+ throws AlgebricksException {
if (paramType.equalsIgnoreCase(BuiltinType.AINT32.getDisplayName())) {
return (BuiltinType.AINT32);
} else if (paramType.equalsIgnoreCase(BuiltinType.AFLOAT.getDisplayName())) {
@@ -112,7 +112,7 @@
}
private static IAType getCollectionType(String paramType, MetadataTransactionContext txnCtx, Function function)
- throws MetadataException {
+ throws AlgebricksException {
Matcher matcher = orderedListPattern.matcher(paramType);
if (matcher.find()) {
@@ -129,7 +129,7 @@
}
private static IResultTypeComputer getResultTypeComputer(final MetadataTransactionContext txnCtx,
- final Function function) throws MetadataException {
+ final Function function) throws AlgebricksException {
final IAType type = getTypeInfo(function.getReturnType(), txnCtx, function);
switch (type.getTypeTag()) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
index 53c3dc0..2c457a9 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
@@ -37,7 +37,6 @@
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.common.transactions.IRecoveryManager;
import org.apache.asterix.common.transactions.JobId;
import org.apache.asterix.external.indexing.IndexingConstants;
@@ -293,7 +292,8 @@
LOGGER.info("CREATING File Splits: " + sb.toString());
Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
- //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
+ // prepare a LocalResourceMetadata which will be stored in NC's local resource
+ // repository
IResourceFactory resourceFactory = dataset.getResourceFactory(metadataProvider, index, itemType, metaItemType,
compactionInfo.first, compactionInfo.second);
IndexBuilderFactory indexBuilderFactory =
@@ -393,74 +393,69 @@
int numFilterFields = DatasetUtil.getFilterField(dataset) == null ? 0 : 1;
ARecordType itemType = (ARecordType) metadataProvider.findType(dataset);
ARecordType metaItemType = (ARecordType) metadataProvider.findMetaType(dataset);
- try {
- Index primaryIndex = metadataProvider.getIndex(dataset.getDataverseName(), dataset.getDatasetName(),
- dataset.getDatasetName());
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
- metadataProvider.getSplitProviderAndConstraints(dataset);
+ Index primaryIndex = metadataProvider.getIndex(dataset.getDataverseName(), dataset.getDatasetName(),
+ dataset.getDatasetName());
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+ metadataProvider.getSplitProviderAndConstraints(dataset);
- // prepare callback
- JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
- int[] primaryKeyFields = new int[numKeys];
- for (int i = 0; i < numKeys; i++) {
- primaryKeyFields[i] = i;
- }
- boolean hasSecondaries =
- metadataProvider.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName()).size() > 1;
- IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
- IModificationOperationCallbackFactory modificationCallbackFactory = dataset.getModificationCallbackFactory(
- storageComponentProvider, primaryIndex, jobId, IndexOperation.UPSERT, primaryKeyFields);
- ISearchOperationCallbackFactory searchCallbackFactory = dataset.getSearchCallbackFactory(
- storageComponentProvider, primaryIndex, jobId, IndexOperation.UPSERT, primaryKeyFields);
- IIndexDataflowHelperFactory idfh = new IndexDataflowHelperFactory(
- storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
- LSMPrimaryUpsertOperatorDescriptor op;
- ITypeTraits[] outputTypeTraits = new ITypeTraits[inputRecordDesc.getFieldCount()
- + (dataset.hasMetaPart() ? 2 : 1) + numFilterFields];
- ISerializerDeserializer<?>[] outputSerDes = new ISerializerDeserializer[inputRecordDesc.getFieldCount()
- + (dataset.hasMetaPart() ? 2 : 1) + numFilterFields];
- IDataFormat dataFormat = metadataProvider.getDataFormat();
-
- // add the previous record first
- int f = 0;
- outputSerDes[f] = dataFormat.getSerdeProvider().getSerializerDeserializer(itemType);
- f++;
- // add the previous meta second
- if (dataset.hasMetaPart()) {
- outputSerDes[f] = dataFormat.getSerdeProvider().getSerializerDeserializer(metaItemType);
- outputTypeTraits[f] = dataFormat.getTypeTraitProvider().getTypeTrait(metaItemType);
- f++;
- }
- // add the previous filter third
- int fieldIdx = -1;
- if (numFilterFields > 0) {
- String filterField = DatasetUtil.getFilterField(dataset).get(0);
- String[] fieldNames = itemType.getFieldNames();
- int i = 0;
- for (; i < fieldNames.length; i++) {
- if (fieldNames[i].equals(filterField)) {
- break;
- }
- }
- fieldIdx = i;
- outputTypeTraits[f] =
- dataFormat.getTypeTraitProvider().getTypeTrait(itemType.getFieldTypes()[fieldIdx]);
- outputSerDes[f] =
- dataFormat.getSerdeProvider().getSerializerDeserializer(itemType.getFieldTypes()[fieldIdx]);
- f++;
- }
- for (int j = 0; j < inputRecordDesc.getFieldCount(); j++) {
- outputTypeTraits[j + f] = inputRecordDesc.getTypeTraits()[j];
- outputSerDes[j + f] = inputRecordDesc.getFields()[j];
- }
- RecordDescriptor outputRecordDesc = new RecordDescriptor(outputSerDes, outputTypeTraits);
- op = new LSMPrimaryUpsertOperatorDescriptor(spec, outputRecordDesc, fieldPermutation, idfh,
- missingWriterFactory, modificationCallbackFactory, searchCallbackFactory,
- dataset.getFrameOpCallbackFactory(), numKeys, itemType, fieldIdx, hasSecondaries);
- return new Pair<>(op, splitsAndConstraint.second);
- } catch (MetadataException me) {
- throw new AlgebricksException(me);
+ // prepare callback
+ JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
+ int[] primaryKeyFields = new int[numKeys];
+ for (int i = 0; i < numKeys; i++) {
+ primaryKeyFields[i] = i;
}
+ boolean hasSecondaries =
+ metadataProvider.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName()).size() > 1;
+ IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
+ IModificationOperationCallbackFactory modificationCallbackFactory = dataset.getModificationCallbackFactory(
+ storageComponentProvider, primaryIndex, jobId, IndexOperation.UPSERT, primaryKeyFields);
+ ISearchOperationCallbackFactory searchCallbackFactory = dataset.getSearchCallbackFactory(
+ storageComponentProvider, primaryIndex, jobId, IndexOperation.UPSERT, primaryKeyFields);
+ IIndexDataflowHelperFactory idfh =
+ new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
+ LSMPrimaryUpsertOperatorDescriptor op;
+ ITypeTraits[] outputTypeTraits =
+ new ITypeTraits[inputRecordDesc.getFieldCount() + (dataset.hasMetaPart() ? 2 : 1) + numFilterFields];
+ ISerializerDeserializer<?>[] outputSerDes = new ISerializerDeserializer[inputRecordDesc.getFieldCount()
+ + (dataset.hasMetaPart() ? 2 : 1) + numFilterFields];
+ IDataFormat dataFormat = metadataProvider.getDataFormat();
+
+ // add the previous record first
+ int f = 0;
+ outputSerDes[f] = dataFormat.getSerdeProvider().getSerializerDeserializer(itemType);
+ f++;
+ // add the previous meta second
+ if (dataset.hasMetaPart()) {
+ outputSerDes[f] = dataFormat.getSerdeProvider().getSerializerDeserializer(metaItemType);
+ outputTypeTraits[f] = dataFormat.getTypeTraitProvider().getTypeTrait(metaItemType);
+ f++;
+ }
+ // add the previous filter third
+ int fieldIdx = -1;
+ if (numFilterFields > 0) {
+ String filterField = DatasetUtil.getFilterField(dataset).get(0);
+ String[] fieldNames = itemType.getFieldNames();
+ int i = 0;
+ for (; i < fieldNames.length; i++) {
+ if (fieldNames[i].equals(filterField)) {
+ break;
+ }
+ }
+ fieldIdx = i;
+ outputTypeTraits[f] = dataFormat.getTypeTraitProvider().getTypeTrait(itemType.getFieldTypes()[fieldIdx]);
+ outputSerDes[f] =
+ dataFormat.getSerdeProvider().getSerializerDeserializer(itemType.getFieldTypes()[fieldIdx]);
+ f++;
+ }
+ for (int j = 0; j < inputRecordDesc.getFieldCount(); j++) {
+ outputTypeTraits[j + f] = inputRecordDesc.getTypeTraits()[j];
+ outputSerDes[j + f] = inputRecordDesc.getFields()[j];
+ }
+ RecordDescriptor outputRecordDesc = new RecordDescriptor(outputSerDes, outputTypeTraits);
+ op = new LSMPrimaryUpsertOperatorDescriptor(spec, outputRecordDesc, fieldPermutation, idfh,
+ missingWriterFactory, modificationCallbackFactory, searchCallbackFactory,
+ dataset.getFrameOpCallbackFactory(), numKeys, itemType, fieldIdx, hasSecondaries);
+ return new Pair<>(op, splitsAndConstraint.second);
}
/**
@@ -503,7 +498,7 @@
}
public static boolean isFullyQualifiedName(String datasetName) {
- return datasetName.indexOf('.') > 0; //NOSONAR a fully qualified name can't start with a .
+ return datasetName.indexOf('.') > 0; // NOSONAR a fully qualified name can't start with a .
}
public static String getFullyQualifiedName(Dataset dataset) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
index 5b7ea59..98cfc57 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
@@ -25,7 +25,6 @@
import org.apache.asterix.common.cluster.ClusterPartition;
import org.apache.asterix.common.cluster.IClusterStateManager;
import org.apache.asterix.common.config.ClusterProperties;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.MetadataTransactionContext;
@@ -59,16 +58,12 @@
public static FileSplit[] getIndexSplits(IClusterStateManager clusterStateManager, Dataset dataset,
String indexName, MetadataTransactionContext mdTxnCtx) throws AlgebricksException {
- try {
- NodeGroup nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, dataset.getNodeGroupName());
- if (nodeGroup == null) {
- throw new AlgebricksException("Couldn't find node group " + dataset.getNodeGroupName());
- }
- List<String> nodeList = nodeGroup.getNodeNames();
- return getIndexSplits(clusterStateManager, dataset, indexName, nodeList);
- } catch (MetadataException me) {
- throw new AlgebricksException(me);
+ NodeGroup nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, dataset.getNodeGroupName());
+ if (nodeGroup == null) {
+ throw new AlgebricksException("Couldn't find node group " + dataset.getNodeGroupName());
}
+ List<String> nodeList = nodeGroup.getNodeNames();
+ return getIndexSplits(clusterStateManager, dataset, indexName, nodeList);
}
public static FileSplit[] getIndexSplits(IClusterStateManager clusterStateManager, Dataset dataset,
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java
index ff65994..40623bd 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java
@@ -25,6 +25,7 @@
import org.apache.asterix.common.transactions.JobId;
import org.apache.asterix.metadata.api.IMetadataEntityTupleTranslator;
import org.apache.asterix.metadata.api.IValueExtractor;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -40,7 +41,7 @@
@Override
public T getValue(JobId jobId, ITupleReference tuple)
- throws MetadataException, HyracksDataException, RemoteException {
+ throws AlgebricksException, HyracksDataException, RemoteException {
return tupleReaderWriter.getMetadataEntityFromTuple(tuple);
}
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
index 5f0525b..22aea26 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
@@ -24,9 +24,9 @@
import java.io.DataInputStream;
import java.io.IOException;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.common.transactions.JobId;
import org.apache.asterix.metadata.api.IValueExtractor;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.util.string.UTF8StringReader;
@@ -49,7 +49,7 @@
private final UTF8StringReader reader = new UTF8StringReader();
@Override
- public String getValue(JobId jobId, ITupleReference tuple) throws MetadataException, HyracksDataException {
+ public String getValue(JobId jobId, ITupleReference tuple) throws AlgebricksException, HyracksDataException {
byte[] serRecord = tuple.getFieldData(2);
int recordStartOffset = tuple.getFieldStart(2);
int recordLength = tuple.getFieldLength(2);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
index 1928d7e..5f16543 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
@@ -21,9 +21,9 @@
import java.nio.ByteBuffer;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.common.transactions.JobId;
import org.apache.asterix.metadata.api.IValueExtractor;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -48,7 +48,8 @@
}
@Override
- public ITupleReference getValue(JobId jobId, ITupleReference tuple) throws MetadataException, HyracksDataException {
+ public ITupleReference getValue(JobId jobId, ITupleReference tuple)
+ throws AlgebricksException, HyracksDataException {
int numBytes = tupleWriter.bytesRequired(tuple);
tupleBytes = new byte[numBytes];
tupleWriter.writeTuple(tuple, tupleBytes, 0);
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
index e5d0d7d..0198d74 100644
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
@@ -30,6 +30,7 @@
import org.apache.asterix.metadata.entities.InternalDatasetDetails.FileStructure;
import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.junit.Assert;
import org.junit.Test;
@@ -37,7 +38,7 @@
public class DatasetTupleTranslatorTest {
@Test
- public void test() throws MetadataException, IOException {
+ public void test() throws AlgebricksException, IOException {
Integer[] indicators = { 0, 1, null };
for (Integer indicator : indicators) {
Map<String, String> compactionPolicyProperties = new HashMap<>();
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
index cdbaad3..32f65e1 100644
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
@@ -41,6 +41,7 @@
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.junit.Assert;
import org.junit.Test;
@@ -48,7 +49,7 @@
public class IndexTupleTranslatorTest {
@Test
- public void test() throws MetadataException, IOException {
+ public void test() throws AlgebricksException, IOException {
Integer[] indicators = { 0, 1, null };
for (Integer indicator : indicators) {
Map<String, String> compactionPolicyProperties = new HashMap<>();
diff --git a/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/datagen/AdmDataGen.java b/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/datagen/AdmDataGen.java
index 70bf864..818444e 100644
--- a/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/datagen/AdmDataGen.java
+++ b/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/datagen/AdmDataGen.java
@@ -46,8 +46,6 @@
import org.apache.asterix.common.annotations.TypeDataGen;
import org.apache.asterix.common.annotations.UndeclaredFieldsDataGen;
import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.common.transactions.JobId;
import org.apache.asterix.lang.aql.parser.AQLParserFactory;
import org.apache.asterix.lang.aql.parser.ParseException;
@@ -727,8 +725,8 @@
"list-val-file annotation cannot be used for field of type " + ti.getTypeTag());
}
AbstractCollectionType act = (AbstractCollectionType) ti;
- declaredFieldsGenerators[i] = new ListFromArrayGenerator(act, a, lvf.getMin(),
- lvf.getMax());
+ declaredFieldsGenerators[i] =
+ new ListFromArrayGenerator(act, a, lvf.getMin(), lvf.getMax());
break;
}
case VALFILESAMEINDEX: {
@@ -774,9 +772,9 @@
}
switch (fi.getValueType()) {
case INT: {
- declaredFieldsGenerators[i] = new IntIntervalGenerator(
- Integer.parseInt(fi.getMin()), Integer.parseInt(fi.getMax()), prefix,
- suffix);
+ declaredFieldsGenerators[i] =
+ new IntIntervalGenerator(Integer.parseInt(fi.getMin()),
+ Integer.parseInt(fi.getMax()), prefix, suffix);
break;
}
case LONG: {
@@ -785,9 +783,9 @@
break;
}
case DOUBLE: {
- declaredFieldsGenerators[i] = new DoubleIntervalGenerator(
- Double.parseDouble(fi.getMin()), Double.parseDouble(fi.getMax()), prefix,
- suffix);
+ declaredFieldsGenerators[i] =
+ new DoubleIntervalGenerator(Double.parseDouble(fi.getMin()),
+ Double.parseDouble(fi.getMax()), prefix, suffix);
break;
}
default: {
@@ -813,14 +811,14 @@
}
case DATEBETWEENYEARS: {
DateBetweenYearsDataGen dby = (DateBetweenYearsDataGen) rfdg;
- declaredFieldsGenerators[i] = new DateBetweenYearsGenerator(dby.getMinYear(),
- dby.getMaxYear());
+ declaredFieldsGenerators[i] =
+ new DateBetweenYearsGenerator(dby.getMinYear(), dby.getMaxYear());
break;
}
case DATETIMEBETWEENYEARS: {
DatetimeBetweenYearsDataGen dtby = (DatetimeBetweenYearsDataGen) rfdg;
- declaredFieldsGenerators[i] = new DatetimeBetweenYearsGenerator(dtby.getMinYear(),
- dtby.getMaxYear());
+ declaredFieldsGenerators[i] =
+ new DatetimeBetweenYearsGenerator(dtby.getMinYear(), dtby.getMaxYear());
break;
}
case DATETIMEADDRANDHOURS: {
@@ -842,21 +840,21 @@
throw new Exception("Couldn't find field " + dtarh.getAddToField() + " before field "
+ recType.getFieldNames()[i]);
}
- declaredFieldsGenerators[i] = new DatetimeAddRandHoursGenerator(dtarh.getMinHour(),
- dtarh.getMaxHour(), adtg);
+ declaredFieldsGenerators[i] =
+ new DatetimeAddRandHoursGenerator(dtarh.getMinHour(), dtarh.getMaxHour(), adtg);
break;
}
case AUTO: {
AutoDataGen auto = (AutoDataGen) rfdg;
switch (ti.getTypeTag()) {
case INTEGER: {
- declaredFieldsGenerators[i] = new IntAutoGenerator(
- Integer.parseInt(auto.getInitValueStr()));
+ declaredFieldsGenerators[i] =
+ new IntAutoGenerator(Integer.parseInt(auto.getInitValueStr()));
break;
}
case BIGINT: {
- declaredFieldsGenerators[i] = new LongAutoGenerator(
- Long.parseLong(auto.getInitValueStr()));
+ declaredFieldsGenerators[i] =
+ new LongAutoGenerator(Long.parseLong(auto.getInitValueStr()));
break;
}
default: {
@@ -881,9 +879,9 @@
if (!recType.isOpen()) {
throw new Exception("Cannot generate undeclared fields for closed type " + recType);
}
- undeclaredFieldsGenerator = new GenFieldsIntGenerator(declaredFieldsGenerators.length,
- ufdg.getMinUndeclaredFields(), ufdg.getMaxUndeclaredFields(),
- ufdg.getUndeclaredFieldsPrefix());
+ undeclaredFieldsGenerator =
+ new GenFieldsIntGenerator(declaredFieldsGenerators.length, ufdg.getMinUndeclaredFields(),
+ ufdg.getMaxUndeclaredFields(), ufdg.getUndeclaredFieldsPrefix());
}
}
if (undeclaredFieldsGenerator != null) {
@@ -937,8 +935,7 @@
this.outputDir = outputDir;
}
- public void init() throws IOException, ParseException, ACIDException,
- AlgebricksException {
+ public void init() throws IOException, ParseException, ACIDException, AlgebricksException {
FileReader aql = new FileReader(schemaFile);
IParser parser = parserFactory.createParser(aql);
List<Statement> statements = parser.parse();
diff --git a/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/translator/ADGenDmlTranslator.java b/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/translator/ADGenDmlTranslator.java
index 9d60995..c6095bd 100644
--- a/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/translator/ADGenDmlTranslator.java
+++ b/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/translator/ADGenDmlTranslator.java
@@ -23,8 +23,6 @@
import java.util.Map;
import org.apache.asterix.common.annotations.TypeDataGen;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.lang.common.statement.DataverseDecl;
import org.apache.asterix.lang.common.statement.TypeDecl;
@@ -49,8 +47,8 @@
public void translate() throws AlgebricksException {
String defaultDataverse = getDefaultDataverse();
- types = new HashMap<TypeSignature, IAType>();
- typeDataGenMap = new HashMap<TypeSignature, TypeDataGen>();
+ types = new HashMap<>();
+ typeDataGenMap = new HashMap<>();
for (Statement stmt : aqlStatements) {
if (stmt.getKind() == Statement.Kind.TYPE_DECL) {