checkpoint
diff --git a/.gitignore b/.gitignore
index 3707fd7..af99107 100644
--- a/.gitignore
+++ b/.gitignore
@@ -3,6 +3,12 @@
.settings
.project
ClusterControllerService
+rttest
+mdtest
+ittest
+asterix_logs
+build
+bin
asterix-app/rttest
asterix-app/mdtest/
asterix-app/opttest/
diff --git a/asterix-algebra/pom.xml b/asterix-algebra/pom.xml
index 074d693..09db13b 100644
--- a/asterix-algebra/pom.xml
+++ b/asterix-algebra/pom.xml
@@ -106,6 +106,12 @@
<version>0.8.1-SNAPSHOT</version>
<scope>compile</scope>
</dependency>
+ <dependency>
+ <groupId>edu.uci.ics.asterix</groupId>
+ <artifactId>asterix-external-data</artifactId>
+ <version>0.8.1-SNAPSHOT</version>
+ <scope>compile</scope>
+ </dependency>
<dependency>
<groupId>edu.uci.ics.asterix</groupId>
<artifactId>asterix-transactions</artifactId>
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/jobgen/AqlLogicalExpressionJobGen.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/jobgen/AqlLogicalExpressionJobGen.java
index 6a48a17..7fc9bcf 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/jobgen/AqlLogicalExpressionJobGen.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/jobgen/AqlLogicalExpressionJobGen.java
@@ -18,10 +18,13 @@
import org.apache.commons.lang3.mutable.Mutable;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.common.functions.FunctionDescriptorTag;
+import edu.uci.ics.asterix.external.library.ExternalFunctionDescriptorProvider;
import edu.uci.ics.asterix.formats.base.IDataFormat;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IExternalFunctionInfo;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.comparisons.ComparisonEvalFactory;
import edu.uci.ics.asterix.runtime.formats.FormatUtils;
@@ -138,9 +141,18 @@
}
IFunctionDescriptor fd = null;
- AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
- IDataFormat format = FormatUtils.getDefaultFormat();
- fd = format.resolveFunction(expr, env);
+ if (!(expr.getFunctionInfo() instanceof IExternalFunctionInfo)) {
+ AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
+ IDataFormat format = FormatUtils.getDefaultFormat();
+ fd = format.resolveFunction(expr, env);
+ } else {
+ try {
+ fd = ExternalFunctionDescriptorProvider.getExternalFunctionDescriptor((IExternalFunctionInfo) expr
+ .getFunctionInfo());
+ } catch (AsterixException ae) {
+ throw new AlgebricksException(ae);
+ }
+ }
return fd.createEvaluatorFactory(args);
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
index ffde764..6b4cdb1 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
@@ -48,6 +48,7 @@
import edu.uci.ics.asterix.optimizer.rules.PushProperJoinThroughProduct;
import edu.uci.ics.asterix.optimizer.rules.PushSimilarityFunctionsBelowJoin;
import edu.uci.ics.asterix.optimizer.rules.RemoveRedundantListifyRule;
+import edu.uci.ics.asterix.optimizer.rules.RemoveSortInFeedIngestionRule;
import edu.uci.ics.asterix.optimizer.rules.RemoveUnusedOneToOneEquiJoinRule;
import edu.uci.ics.asterix.optimizer.rules.ReplaceSinkOpWithCommitOpRule;
import edu.uci.ics.asterix.optimizer.rules.SetAsterixPhysicalOperatorsRule;
@@ -244,6 +245,7 @@
physicalRewritesAllLevels.add(new SetAsterixPhysicalOperatorsRule());
physicalRewritesAllLevels.add(new IntroduceInstantLockSearchCallbackRule());
physicalRewritesAllLevels.add(new EnforceStructuralPropertiesRule());
+ physicalRewritesAllLevels.add(new RemoveSortInFeedIngestionRule());
physicalRewritesAllLevels.add(new IntroHashPartitionMergeExchange());
physicalRewritesAllLevels.add(new SetClosedRecordConstructorsRule());
physicalRewritesAllLevels.add(new PullPositionalVariableFromUnnestRule());
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java
index 62cca6c..a2b8ef5 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java
@@ -25,6 +25,7 @@
import edu.uci.ics.asterix.algebra.operators.physical.BTreeSearchPOperator;
import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataImplConfig;
+import edu.uci.ics.asterix.metadata.declared.DatasetDataSource;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.optimizer.rules.am.AccessMethodJobGenParams;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -81,11 +82,11 @@
}
} else if (descendantOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
DataSourceScanOperator dataSourceScanOp = (DataSourceScanOperator) descendantOp;
- String datasetName = ((AqlDataSource) dataSourceScanOp.getDataSource()).getDataset().getDatasetName();
- if (dataSourcesMap.containsKey(datasetName)) {
- ++(dataSourcesMap.get(datasetName).first);
+ String datasourceName = ((AqlDataSource) dataSourceScanOp.getDataSource()).getDatasourceName();
+ if (dataSourcesMap.containsKey(datasourceName)) {
+ ++(dataSourcesMap.get(datasourceName).first);
} else {
- dataSourcesMap.put(datasetName, new Triple<Integer, LogicalOperatorTag, IPhysicalOperator>(1,
+ dataSourcesMap.put(datasourceName, new Triple<Integer, LogicalOperatorTag, IPhysicalOperator>(1,
LogicalOperatorTag.DATASOURCESCAN, dataSourceScanOp.getPhysicalOperator()));
}
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java
index d33d674..301c685 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java
@@ -25,6 +25,8 @@
import edu.uci.ics.asterix.algebra.base.AsterixOperatorAnnotations;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.exceptions.AsterixRuntimeException;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
import edu.uci.ics.asterix.metadata.entities.Dataset;
@@ -129,11 +131,12 @@
}
AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
AqlSourceId asid = ((IDataSource<AqlSourceId>) scan.getDataSource()).getId();
+
Dataset dataset = mp.findDataset(asid.getDataverseName(), asid.getDatasetName());
if (dataset == null) {
throw new AlgebricksException("Dataset " + asid.getDatasetName() + " not found.");
}
- if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
+ if (dataset.getDatasetType() != DatasetType.INTERNAL) {
return false;
}
ILogicalExpression e1 = accessFun.getArguments().get(1).getValue();
@@ -302,14 +305,16 @@
ILogicalExpression e1 = accessFun.getArguments().get(1).getValue();
if (e1.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
IDataSource<AqlSourceId> dataSource = (IDataSource<AqlSourceId>) scan.getDataSource();
+ if (((AqlDataSource) dataSource).getDatasourceType().equals(AqlDataSourceType.FEED)) {
+ return false;
+ }
AqlSourceId asid = dataSource.getId();
AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
Dataset dataset = mp.findDataset(asid.getDataverseName(), asid.getDatasetName());
if (dataset == null) {
throw new AlgebricksException("Dataset " + asid.getDatasetName() + " not found.");
}
- if (dataset.getDatasetType() != DatasetType.INTERNAL
- && dataset.getDatasetType() != DatasetType.FEED) {
+ if (dataset.getDatasetType() != DatasetType.INTERNAL) {
setAsFinal(access, context, finalAnnot);
return false;
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/RemoveSortInFeedIngestionRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/RemoveSortInFeedIngestionRule.java
new file mode 100644
index 0000000..951cacf
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/RemoveSortInFeedIngestionRule.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.optimizer.rules;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class RemoveSortInFeedIngestionRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.INSERT_DELETE) {
+ return false;
+ }
+
+ AbstractLogicalOperator insertOp = op;
+ AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+ boolean isSourceAFeed = false;
+ while (descendantOp != null) {
+ if (descendantOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+ AqlDataSource dataSource = (AqlDataSource) ((DataSourceScanOperator) descendantOp).getDataSource();
+ if (dataSource.getDatasourceType().equals(AqlDataSourceType.FEED)) {
+ isSourceAFeed = true;
+ }
+ break;
+ }
+ if (descendantOp.getInputs().isEmpty()) {
+ break;
+ }
+ descendantOp = (AbstractLogicalOperator) descendantOp.getInputs().get(0).getValue();
+ }
+
+ if (isSourceAFeed) {
+ AbstractLogicalOperator prevOp = (AbstractLogicalOperator) insertOp.getInputs().get(0).getValue();
+ if (prevOp.getOperatorTag() == LogicalOperatorTag.ORDER) {
+ insertOp.getInputs().set(0, prevOp.getInputs().get(0));
+ return true;
+ }
+ }
+
+ return false;
+ }
+
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
index 9fe5e46..09715f5 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
@@ -23,6 +23,7 @@
import org.apache.commons.lang3.mutable.Mutable;
import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
+import edu.uci.ics.asterix.metadata.declared.DatasetDataSource;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -45,14 +46,13 @@
* Removes join operators for which all of the following conditions are true:
* 1. The live variables of one input branch of the join are not used in the upstream plan
* 2. The join is an inner equi join
- * 3. The join condition only uses variables that correspond to primary keys of the same dataset
+ * 3. The join condition only uses variables that correspond to primary keys of the same dataset
* Notice that the last condition implies a 1:1 join, i.e., the join does not change the result cardinality.
- *
- * Joins that satisfy the above conditions may be introduced by other rules
+ * Joins that satisfy the above conditions may be introduced by other rules
* which use surrogate optimizations. Such an optimization aims to reduce data copies and communication costs by
* using the primary keys as surrogates for the desired data items. Typically,
* such a surrogate-based plan introduces a top-level join to finally resolve
- * the surrogates to the desired data items.
+ * the surrogates to the desired data items.
* In case the upstream plan does not require the original data items at all, such a top-level join is unnecessary.
* The purpose of this rule is to remove such unnecessary joins.
*/
@@ -152,8 +152,8 @@
// only used primary key variables of those datascans.
for (int i = 0; i < dataScans.size(); i++) {
if (i > 0) {
- AqlDataSource prevAqlDataSource = (AqlDataSource) dataScans.get(i - 1).getDataSource();
- AqlDataSource currAqlDataSource = (AqlDataSource) dataScans.get(i).getDataSource();
+ DatasetDataSource prevAqlDataSource = (DatasetDataSource) dataScans.get(i - 1).getDataSource();
+ DatasetDataSource currAqlDataSource = (DatasetDataSource) dataScans.get(i).getDataSource();
if (!prevAqlDataSource.getDataset().equals(currAqlDataSource.getDataset())) {
return -1;
}
@@ -189,8 +189,8 @@
private void fillPKVars(DataSourceScanOperator dataScan, List<LogicalVariable> pkVars) {
pkVars.clear();
- AqlDataSource aqlDataSource = (AqlDataSource) dataScan.getDataSource();
- int numPKs = DatasetUtils.getPartitioningKeys(aqlDataSource.getDataset()).size();
+ DatasetDataSource datasetDataSource = (DatasetDataSource) dataScan.getDataSource();
+ int numPKs = DatasetUtils.getPartitioningKeys(datasetDataSource.getDataset()).size();
pkVars.clear();
for (int i = 0; i < numPKs; i++) {
pkVars.add(dataScan.getVariables().get(i));
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
index 1fe7ee9..2550a34 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
@@ -24,6 +24,7 @@
import edu.uci.ics.asterix.common.transactions.JobId;
import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.declared.DatasetDataSource;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
@@ -63,12 +64,13 @@
if (descendantOp.getOperatorTag() == LogicalOperatorTag.INDEX_INSERT_DELETE) {
IndexInsertDeleteOperator indexInsertDeleteOperator = (IndexInsertDeleteOperator) descendantOp;
primaryKeyExprs = indexInsertDeleteOperator.getPrimaryKeyExpressions();
- datasetId = ((AqlDataSource) indexInsertDeleteOperator.getDataSourceIndex().getDataSource()).getDataset().getDatasetId();
+ datasetId = ((DatasetDataSource) indexInsertDeleteOperator.getDataSourceIndex().getDataSource())
+ .getDataset().getDatasetId();
break;
} else if (descendantOp.getOperatorTag() == LogicalOperatorTag.INSERT_DELETE) {
InsertDeleteOperator insertDeleteOperator = (InsertDeleteOperator) descendantOp;
primaryKeyExprs = insertDeleteOperator.getPrimaryKeyExpressions();
- datasetId = ((AqlDataSource) insertDeleteOperator.getDataSource()).getDataset().getDatasetId();
+ datasetId = ((DatasetDataSource) insertDeleteOperator.getDataSource()).getDataset().getDatasetId();
break;
}
descendantOp = (AbstractLogicalOperator) descendantOp.getInputs().get(0).getValue();
@@ -79,7 +81,7 @@
//copy primaryKeyExprs
List<LogicalVariable> primaryKeyLogicalVars = new ArrayList<LogicalVariable>();
for (Mutable<ILogicalExpression> expr : primaryKeyExprs) {
- VariableReferenceExpression varRefExpr = (VariableReferenceExpression)expr.getValue();
+ VariableReferenceExpression varRefExpr = (VariableReferenceExpression) expr.getValue();
primaryKeyLogicalVars.add(new LogicalVariable(varRefExpr.getVariableReference().getId()));
}
@@ -89,13 +91,14 @@
//create the logical and physical operator
CommitOperator commitOperator = new CommitOperator(primaryKeyLogicalVars);
- CommitPOperator commitPOperator = new CommitPOperator(jobId, datasetId, primaryKeyLogicalVars, mp.isWriteTransaction());
+ CommitPOperator commitPOperator = new CommitPOperator(jobId, datasetId, primaryKeyLogicalVars,
+ mp.isWriteTransaction());
commitOperator.setPhysicalOperator(commitPOperator);
//create ExtensionOperator and put the commitOperator in it.
ExtensionOperator extensionOperator = new ExtensionOperator(commitOperator);
extensionOperator.setPhysicalOperator(commitPOperator);
-
+
//update plan link
extensionOperator.getInputs().add(sinkOperator.getInputs().get(0));
context.computeAndSetTypeEnvironmentForOperator(extensionOperator);
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java
index 642ef22..c087bd1 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java
@@ -23,13 +23,20 @@
import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
-import edu.uci.ics.asterix.metadata.declared.ExternalFeedDataSource;
+import edu.uci.ics.asterix.metadata.declared.FeedDataSource;
import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.Feed;
+import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
+import edu.uci.ics.asterix.metadata.feeds.BuiltinFeedPolicies;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.metadata.feeds.FeedUtil;
+import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
import edu.uci.ics.asterix.om.base.AString;
import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.ATypeTag;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -104,14 +111,13 @@
ArrayList<LogicalVariable> v = new ArrayList<LogicalVariable>();
- if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
+ if (dataset.getDatasetType() == DatasetType.INTERNAL) {
int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
for (int i = 0; i < numPrimaryKeys; i++) {
v.add(context.newVar());
}
}
v.add(unnest.getVariable());
-
DataSourceScanOperator scan = new DataSourceScanOperator(v, metadataProvider.findDataSource(asid));
List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
scanInpList.addAll(unnest.getInputs());
@@ -126,40 +132,36 @@
if (unnest.getPositionalVariable() != null) {
throw new AlgebricksException("No positional variables are allowed over datasets.");
}
- ILogicalExpression expr = f.getArguments().get(0).getValue();
- if (expr.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
- return false;
- }
- ConstantExpression ce = (ConstantExpression) expr;
- IAlgebricksConstantValue acv = ce.getValue();
- if (!(acv instanceof AsterixConstantValue)) {
- return false;
- }
- AsterixConstantValue acv2 = (AsterixConstantValue) acv;
- if (acv2.getObject().getType().getTypeTag() != ATypeTag.STRING) {
- return false;
- }
- String datasetArg = ((AString) acv2.getObject()).getStringValue();
+
+ String feedArg = getStringArgument(f, 0);
+ String outputType = getStringArgument(f, 1);
+ String targetDataset = getStringArgument(f, 2);
AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
- Pair<String, String> datasetReference = parseDatasetReference(metadataProvider, datasetArg);
- String dataverseName = datasetReference.first;
- String datasetName = datasetReference.second;
- Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
- if (dataset == null) {
- throw new AlgebricksException("Could not find dataset " + datasetName);
+ Pair<String, String> feedReference = parseDatasetReference(metadataProvider, feedArg);
+ String dataverseName = feedReference.first;
+ String feedName = feedReference.second;
+ Feed feed = metadataProvider.findFeed(dataverseName, feedName);
+ if (feed == null) {
+ throw new AlgebricksException("Could not find feed " + feedName);
}
- if (dataset.getDatasetType() != DatasetType.FEED) {
- throw new IllegalArgumentException("invalid dataset type:" + dataset.getDatasetType());
+ AqlSourceId asid = new AqlSourceId(dataverseName, feedName);
+ String policyName = metadataProvider.getConfig().get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY);
+ FeedPolicy policy = metadataProvider.findFeedPolicy(dataverseName, policyName);
+ if (policy == null) {
+ policy = BuiltinFeedPolicies.getFeedPolicy(policyName);
+ if (policy == null) {
+ throw new AlgebricksException("Unknown feed policy:" + policyName);
+ }
}
- AqlSourceId asid = new AqlSourceId(dataverseName, datasetName);
ArrayList<LogicalVariable> v = new ArrayList<LogicalVariable>();
v.add(unnest.getVariable());
- DataSourceScanOperator scan = new DataSourceScanOperator(v, createDummyFeedDataSource(asid, dataset,
- metadataProvider));
+ DataSourceScanOperator scan = new DataSourceScanOperator(v, createFeedDataSource(asid,
+ new FeedConnectionId(dataverseName, feedName, targetDataset), metadataProvider, policy,
+ outputType));
List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
scanInpList.addAll(unnest.getInputs());
@@ -169,6 +171,7 @@
return true;
}
+
}
return false;
@@ -183,18 +186,18 @@
context.addPrimaryKey(pk);
}
- private AqlDataSource createDummyFeedDataSource(AqlSourceId aqlId, Dataset dataset,
- AqlMetadataProvider metadataProvider) throws AlgebricksException {
+ private AqlDataSource createFeedDataSource(AqlSourceId aqlId, FeedConnectionId feedId,
+ AqlMetadataProvider metadataProvider, FeedPolicy feedPolicy, String outputType) throws AlgebricksException {
if (!aqlId.getDataverseName().equals(
metadataProvider.getDefaultDataverse() == null ? null : metadataProvider.getDefaultDataverse()
.getDataverseName())) {
return null;
}
- String tName = dataset.getItemTypeName();
- IAType itemType = metadataProvider.findType(dataset.getDataverseName(), tName);
- ExternalFeedDataSource extDataSource = new ExternalFeedDataSource(aqlId, dataset, itemType,
- AqlDataSource.AqlDataSourceType.EXTERNAL_FEED);
- return extDataSource;
+ IAType feedOutputType = metadataProvider.findType(feedId.getDataverse(), outputType);
+ FeedDataSource feedDataSource = new FeedDataSource(aqlId, feedId, feedOutputType,
+ AqlDataSource.AqlDataSourceType.FEED);
+ feedDataSource.getProperties().put(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY, feedPolicy);
+ return feedDataSource;
}
private Pair<String, String> parseDatasetReference(AqlMetadataProvider metadataProvider, String datasetArg)
@@ -215,4 +218,23 @@
}
return new Pair<String, String>(dataverseName, datasetName);
}
+
+ private String getStringArgument(AbstractFunctionCallExpression f, int index) {
+
+ ILogicalExpression expr = f.getArguments().get(index).getValue();
+ if (expr.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+ return null;
+ }
+ ConstantExpression ce = (ConstantExpression) expr;
+ IAlgebricksConstantValue acv = ce.getValue();
+ if (!(acv instanceof AsterixConstantValue)) {
+ return null;
+ }
+ AsterixConstantValue acv2 = (AsterixConstantValue) acv;
+ if (acv2.getObject().getType().getTypeTag() != ATypeTag.STRING) {
+ return null;
+ }
+ String argument = ((AString) acv2.getObject()).getStringValue();
+ return argument;
+ }
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index 28aee7a..334d411 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -114,7 +114,7 @@
if (dataset == null) {
throw new AlgebricksException("No metadata for dataset " + datasetName);
}
- if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
+ if (dataset.getDatasetType() != DatasetType.INTERNAL) {
return false;
}
// Get the record type for that dataset.
@@ -129,7 +129,7 @@
public boolean hasDataSourceScan() {
return dataSourceScan != null;
}
-
+
public void reset() {
root = null;
rootRef = null;
@@ -140,7 +140,7 @@
dataset = null;
recordType = null;
}
-
+
public void getPrimaryKeyVars(List<LogicalVariable> target) {
int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
for (int i = 0; i < numPrimaryKeys; i++) {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
index c67a4e5..d25dba3 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
@@ -223,7 +223,8 @@
boolean changed = false;
for (int j = 0; j < args.size(); j++) {
ILogicalExpression arg = args.get(j).getValue();
- IAType currentItemType = (inputItemType == null || inputItemType == BuiltinType.ANY) ? (IAType) env.getType(arg) : inputItemType;
+ IAType currentItemType = (inputItemType == null || inputItemType == BuiltinType.ANY) ? (IAType) env
+ .getType(arg) : inputItemType;
switch (arg.getExpressionTag()) {
case FUNCTION_CALL:
ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) arg;
@@ -252,6 +253,10 @@
*/
private static boolean staticRecordTypeCast(AbstractFunctionCallExpression func, ARecordType reqType,
ARecordType inputType, IVariableTypeEnvironment env) throws AlgebricksException {
+ if (!(func.getFunctionIdentifier() == AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR || func
+ .getFunctionIdentifier() == AsterixBuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR)) {
+ return false;
+ }
IAType[] reqFieldTypes = reqType.getFieldTypes();
String[] reqFieldNames = reqType.getFieldNames();
IAType[] inputFieldTypes = inputType.getFieldTypes();
@@ -463,12 +468,9 @@
reqFieldType = DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
fi = AsterixBuiltinFunctions.CAST_LIST;
}
- if (fi != null
- && ! inputFieldType.equals(reqFieldType)
- && parameterVars.size() > 0) {
+ if (fi != null && !inputFieldType.equals(reqFieldType) && parameterVars.size() > 0) {
//inject dynamic type casting
- injectCastFunction(FunctionUtils.getFunctionInfo(fi),
- reqFieldType, inputFieldType, expRef, argExpr);
+ injectCastFunction(FunctionUtils.getFunctionInfo(fi), reqFieldType, inputFieldType, expRef, argExpr);
castInjected = true;
}
if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
index 7b12550..31095f9 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
@@ -26,20 +26,22 @@
import edu.uci.ics.asterix.aql.base.Clause;
import edu.uci.ics.asterix.aql.base.Expression;
import edu.uci.ics.asterix.aql.base.Expression.Kind;
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
import edu.uci.ics.asterix.aql.expression.CallExpr;
import edu.uci.ics.asterix.aql.expression.CompactStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
import edu.uci.ics.asterix.aql.expression.DatasetDecl;
import edu.uci.ics.asterix.aql.expression.DataverseDecl;
import edu.uci.ics.asterix.aql.expression.DataverseDropStatement;
import edu.uci.ics.asterix.aql.expression.DeleteStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.DistinctClause;
import edu.uci.ics.asterix.aql.expression.DropStatement;
import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
+import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
import edu.uci.ics.asterix.aql.expression.FieldAccessor;
import edu.uci.ics.asterix.aql.expression.FieldBinding;
import edu.uci.ics.asterix.aql.expression.ForClause;
@@ -93,13 +95,15 @@
import edu.uci.ics.asterix.formats.base.IDataFormat;
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.MetadataManager;
-import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
+import edu.uci.ics.asterix.metadata.declared.DatasetDataSource;
import edu.uci.ics.asterix.metadata.declared.ResultSetDataSink;
import edu.uci.ics.asterix.metadata.declared.ResultSetSinkId;
import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.entities.Function;
+import edu.uci.ics.asterix.metadata.functions.ExternalFunctionCompilerUtil;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
import edu.uci.ics.asterix.om.base.AInt32;
import edu.uci.ics.asterix.om.base.AString;
@@ -153,7 +157,6 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
@@ -231,7 +234,7 @@
project.getVariables().set(0, seqVar);
resVar = seqVar;
- AqlDataSource targetDatasource = validateDatasetInfo(metadataProvider, stmt.getDataverseName(),
+ DatasetDataSource targetDatasource = validateDatasetInfo(metadataProvider, stmt.getDataverseName(),
stmt.getDatasetName());
ArrayList<LogicalVariable> vars = new ArrayList<LogicalVariable>();
ArrayList<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
@@ -274,7 +277,7 @@
leafOperator.getInputs().add(new MutableObject<ILogicalOperator>(deleteOp));
break;
}
- case BEGIN_FEED: {
+ case CONNECT_FEED: {
ILogicalOperator insertOp = new InsertDeleteOperator(targetDatasource, varRef, varRefsForLoading,
InsertDeleteOperator.Kind.INSERT);
insertOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
@@ -290,20 +293,21 @@
return plan;
}
- private AqlDataSource validateDatasetInfo(AqlMetadataProvider metadataProvider, String dataverseName,
+ private DatasetDataSource validateDatasetInfo(AqlMetadataProvider metadataProvider, String dataverseName,
String datasetName) throws AlgebricksException {
Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
if (dataset == null) {
throw new AlgebricksException("Cannot find dataset " + datasetName + " in dataverse " + dataverseName);
}
-
- AqlSourceId sourceId = new AqlSourceId(dataverseName, datasetName);
- String itemTypeName = dataset.getItemTypeName();
- IAType itemType = metadataProvider.findType(dataverseName, itemTypeName);
- AqlDataSource dataSource = new AqlDataSource(sourceId, dataset, itemType);
if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
throw new AlgebricksException("Cannot write output to an external dataset.");
}
+ AqlSourceId sourceId = new AqlSourceId(dataverseName, datasetName);
+ String itemTypeName = dataset.getItemTypeName();
+ IAType itemType = metadataProvider.findType(dataverseName, itemTypeName);
+ DatasetDataSource dataSource = new DatasetDataSource(sourceId, dataset.getDataverseName(),
+ dataset.getDatasetName(), itemType, AqlDataSourceType.INTERNAL_DATASET);
+
return dataSource;
}
@@ -509,13 +513,18 @@
return null;
}
AbstractFunctionCallExpression f = null;
- if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)) {
+ if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_JAVA)) {
+ IFunctionInfo finfo = ExternalFunctionCompilerUtil.getExternalFunctionInfo(
+ metadataProvider.getMetadataTxnContext(), function);
+ f = new ScalarFunctionCallExpression(finfo, args);
+ } else if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)) {
IFunctionInfo finfo = new AsterixFunctionInfo(signature);
- return new ScalarFunctionCallExpression(finfo, args);
+ f = new ScalarFunctionCallExpression(finfo, args);
} else {
throw new MetadataException(" User defined functions written in " + function.getLanguage()
+ " are not supported");
}
+ return f;
}
private AbstractFunctionCallExpression lookupBuiltinFunction(String functionName, int arity,
@@ -1270,7 +1279,7 @@
}
@Override
- public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(ControlFeedStatement del,
+ public Pair<ILogicalOperator, LogicalVariable> visitDisconnectFeedStatement(DisconnectFeedStatement del,
Mutable<ILogicalOperator> arg) throws AsterixException {
// TODO Auto-generated method stub
return null;
@@ -1426,7 +1435,21 @@
}
@Override
- public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(BeginFeedStatement bf,
+ public Pair<ILogicalOperator, LogicalVariable> visitCreateFeedStatement(CreateFeedStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitConnectFeedStatement(ConnectFeedStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitDropFeedStatement(FeedDropStatement del,
Mutable<ILogicalOperator> arg) throws AsterixException {
// TODO Auto-generated method stub
return null;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
index 4d11dbd..5c4126f 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
@@ -26,11 +26,12 @@
import edu.uci.ics.asterix.aql.base.Clause;
import edu.uci.ics.asterix.aql.base.Expression;
import edu.uci.ics.asterix.aql.base.Expression.Kind;
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
import edu.uci.ics.asterix.aql.expression.CallExpr;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.CompactStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
import edu.uci.ics.asterix.aql.expression.DatasetDecl;
@@ -40,6 +41,7 @@
import edu.uci.ics.asterix.aql.expression.DistinctClause;
import edu.uci.ics.asterix.aql.expression.DropStatement;
import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
+import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
import edu.uci.ics.asterix.aql.expression.FieldAccessor;
import edu.uci.ics.asterix.aql.expression.FieldBinding;
import edu.uci.ics.asterix.aql.expression.ForClause;
@@ -1423,7 +1425,7 @@
}
@Override
- public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(ControlFeedStatement del,
+ public Pair<ILogicalOperator, LogicalVariable> visitDisconnectFeedStatement(DisconnectFeedStatement del,
Mutable<ILogicalOperator> arg) throws AsterixException {
// TODO Auto-generated method stub
return null;
@@ -1444,7 +1446,21 @@
}
@Override
- public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(BeginFeedStatement bf,
+ public Pair<ILogicalOperator, LogicalVariable> visitCreateFeedStatement(CreateFeedStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitConnectFeedStatement(ConnectFeedStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitDropFeedStatement(FeedDropStatement del,
Mutable<ILogicalOperator> arg) throws AsterixException {
// TODO Auto-generated method stub
return null;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/CompiledStatements.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/CompiledStatements.java
index d9aa047..486aeee 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/CompiledStatements.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/CompiledStatements.java
@@ -22,7 +22,6 @@
import edu.uci.ics.asterix.aql.base.Expression;
import edu.uci.ics.asterix.aql.base.Statement.Kind;
import edu.uci.ics.asterix.aql.expression.CallExpr;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement.OperationType;
import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
import edu.uci.ics.asterix.aql.expression.FieldAccessor;
import edu.uci.ics.asterix.aql.expression.FieldBinding;
@@ -321,15 +320,20 @@
}
}
- public static class CompiledBeginFeedStatement implements ICompiledDmlStatement {
+ public static class CompiledConnectFeedStatement implements ICompiledDmlStatement {
private String dataverseName;
+ private String feedName;
private String datasetName;
+ private String policyName;
private Query query;
private int varCounter;
- public CompiledBeginFeedStatement(String dataverseName, String datasetName, Query query, int varCounter) {
+ public CompiledConnectFeedStatement(String dataverseName, String feedName, String datasetName,
+ String policyName, Query query, int varCounter) {
this.dataverseName = dataverseName;
+ this.feedName = feedName;
this.datasetName = datasetName;
+ this.policyName = policyName;
this.query = query;
this.varCounter = varCounter;
}
@@ -339,6 +343,10 @@
return dataverseName;
}
+ public String getFeedName() {
+ return feedName;
+ }
+
@Override
public String getDatasetName() {
return datasetName;
@@ -358,24 +366,25 @@
@Override
public Kind getKind() {
- return Kind.BEGIN_FEED;
+ return Kind.CONNECT_FEED;
+ }
+
+ public String getPolicyName() {
+ return policyName;
}
}
- public static class CompiledControlFeedStatement implements ICompiledDmlStatement {
+ public static class CompiledDisconnectFeedStatement implements ICompiledDmlStatement {
private String dataverseName;
private String datasetName;
- private OperationType operationType;
+ private String feedName;
private Query query;
private int varCounter;
- private Map<String, String> alteredParams;
- public CompiledControlFeedStatement(OperationType operationType, String dataverseName, String datasetName,
- Map<String, String> alteredParams) {
+ public CompiledDisconnectFeedStatement(String dataverseName, String feedName, String datasetName) {
this.dataverseName = dataverseName;
+ this.feedName = feedName;
this.datasetName = datasetName;
- this.operationType = operationType;
- this.alteredParams = alteredParams;
}
@Override
@@ -388,8 +397,8 @@
return datasetName;
}
- public OperationType getOperationType() {
- return operationType;
+ public String getFeedName() {
+ return feedName;
}
public int getVarCounter() {
@@ -402,16 +411,9 @@
@Override
public Kind getKind() {
- return Kind.CONTROL_FEED;
+ return Kind.DISCONNECT_FEED;
}
- public Map<String, String> getProperties() {
- return alteredParams;
- }
-
- public void setProperties(Map<String, String> properties) {
- this.alteredParams = properties;
- }
}
public static class CompiledDeleteStatement implements ICompiledDmlStatement {
diff --git a/asterix-app/.gitignore b/asterix-app/.gitignore
deleted file mode 100644
index ea8c4bf..0000000
--- a/asterix-app/.gitignore
+++ /dev/null
@@ -1 +0,0 @@
-/target
diff --git a/asterix-app/pom.xml b/asterix-app/pom.xml
index e80e33d..06d289e 100644
--- a/asterix-app/pom.xml
+++ b/asterix-app/pom.xml
@@ -1,18 +1,14 @@
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<!-- ! Copyright 2009-2013 by The Regents of the University of California
+ ! Licensed under the Apache License, Version 2.0 (the "License"); ! you may
+ not use this file except in compliance with the License. ! you may obtain
+ a copy of the License from ! ! http://www.apache.org/licenses/LICENSE-2.0
+ ! ! Unless required by applicable law or agreed to in writing, software !
+ distributed under the License is distributed on an "AS IS" BASIS, ! WITHOUT
+ WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. ! See the
+ License for the specific language governing permissions and ! limitations
+ under the License. ! -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<artifactId>asterix</artifactId>
@@ -90,7 +86,7 @@
<additionalClasspathElement>${basedir}/src/main/resources</additionalClasspathElement>
</additionalClasspathElements> -->
<forkMode>pertest</forkMode>
- <argLine>-enableassertions -Xmx${test.heap.size}m
+ <argLine>-enableassertions -Xmx${test.heap.size}m
-Dfile.encoding=UTF-8
-Djava.util.logging.config.file=src/test/resources/logging.properties
-Xdebug
@@ -106,12 +102,6 @@
<dependencies>
<dependency>
- <groupId>edu.uci.ics.asterix</groupId>
- <artifactId>asterix-algebra</artifactId>
- <version>0.8.1-SNAPSHOT</version>
- <scope>compile</scope>
- </dependency>
- <dependency>
<groupId>javax.servlet</groupId>
<artifactId>servlet-api</artifactId>
<type>jar</type>
@@ -149,9 +139,15 @@
<artifactId>algebricks-compiler</artifactId>
</dependency>
<dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-client</artifactId>
- </dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-client</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.asterix</groupId>
+ <artifactId>asterix-algebra</artifactId>
+ <version>0.8.1-SNAPSHOT</version>
+ <scope>compile</scope>
+ </dependency>
<dependency>
<groupId>edu.uci.ics.asterix</groupId>
<artifactId>asterix-aql</artifactId>
@@ -187,19 +183,22 @@
<type>jar</type>
<scope>compile</scope>
</dependency>
- <dependency>
- <groupId>edu.uci.ics.asterix</groupId>
- <artifactId>asterix-common</artifactId>
- <version>0.8.1-SNAPSHOT</version>
- <type>test-jar</type>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.asterix</groupId>
- <artifactId>asterix-transactions</artifactId>
- <version>0.8.1-SNAPSHOT</version>
- <scope>compile</scope>
- </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.asterix</groupId>
+ <artifactId>asterix-common</artifactId>
+ <version>0.8.1-SNAPSHOT</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <!-- posssible remove this <dependency> <groupId>com.kenai.nbpwr</groupId>
+ <artifactId>org-apache-commons-io</artifactId> <version>1.3.1-201002241208</version>
+ <scope>test</scope> </dependency> -->
+ <dependency>
+ <groupId>edu.uci.ics.asterix</groupId>
+ <artifactId>asterix-transactions</artifactId>
+ <version>0.8.1-SNAPSHOT</version>
+ <scope>compile</scope>
+ </dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>
@@ -227,12 +226,12 @@
<type>jar</type>
<scope>test</scope>
</dependency>
- <dependency>
- <groupId>edu.uci.ics.asterix</groupId>
- <artifactId>asterix-test-framework</artifactId>
- <version>0.8.1-SNAPSHOT</version>
- <scope>test</scope>
- </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.asterix</groupId>
+ <artifactId>asterix-test-framework</artifactId>
+ <version>0.8.1-SNAPSHOT</version>
+ <scope>test</scope>
+ </dependency>
</dependencies>
</project>
diff --git a/asterix-app/scripts/asterix/startallncs.sh b/asterix-app/scripts/asterix/startallncs.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/asterix/startcc.sh b/asterix-app/scripts/asterix/startcc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/asterix/startnc.sh b/asterix-app/scripts/asterix/startnc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/asterix/stopallncs.sh b/asterix-app/scripts/asterix/stopallncs.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/asterix/stopcc.sh b/asterix-app/scripts/asterix/stopcc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/asterix/stopnc.sh b/asterix-app/scripts/asterix/stopnc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/asterix/test.properties b/asterix-app/scripts/asterix/test.properties
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/idefix/startcc.sh b/asterix-app/scripts/idefix/startcc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/idefix/startnc1.sh b/asterix-app/scripts/idefix/startnc1.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/idefix/startnc2.sh b/asterix-app/scripts/idefix/startnc2.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/idefix/stopallncs.sh b/asterix-app/scripts/idefix/stopallncs.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/idefix/stopcc.sh b/asterix-app/scripts/idefix/stopcc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/rainbow/execute.sh b/asterix-app/scripts/rainbow/execute.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/rainbow/startallncs.sh b/asterix-app/scripts/rainbow/startallncs.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/rainbow/startcc.sh b/asterix-app/scripts/rainbow/startcc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/rainbow/startnc.sh b/asterix-app/scripts/rainbow/startnc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/rainbow/stopallncs.sh b/asterix-app/scripts/rainbow/stopallncs.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/rainbow/stopcc.sh b/asterix-app/scripts/rainbow/stopcc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/scripts/rainbow/stopnc.sh b/asterix-app/scripts/rainbow/stopnc.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
index 16e3c14..c2c75bc 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
@@ -64,6 +64,9 @@
import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactoryProvider;
public class AsterixAppRuntimeContext implements IAsterixAppRuntimeContext, IAsterixPropertiesProvider {
+
+ public static final AsterixPropertiesAccessor ASTERIX_PROPERTIES_ACCESSOR = createAsterixPropertiesAccessor();
+
private static final int METADATA_IO_DEVICE_ID = 0;
private ILSMMergePolicyFactory metadataMergePolicyFactory;
@@ -86,18 +89,26 @@
private IIOManager ioManager;
private boolean isShuttingdown;
- public AsterixAppRuntimeContext(INCApplicationContext ncApplicationContext) {
+ public AsterixAppRuntimeContext(INCApplicationContext ncApplicationContext) throws AsterixException {
this.ncApplicationContext = ncApplicationContext;
+ compilerProperties = new AsterixCompilerProperties(ASTERIX_PROPERTIES_ACCESSOR);
+ externalProperties = new AsterixExternalProperties(ASTERIX_PROPERTIES_ACCESSOR);
+ metadataProperties = new AsterixMetadataProperties(ASTERIX_PROPERTIES_ACCESSOR);
+ storageProperties = new AsterixStorageProperties(ASTERIX_PROPERTIES_ACCESSOR);
+ txnProperties = new AsterixTransactionProperties(ASTERIX_PROPERTIES_ACCESSOR);
+ }
+
+ private static AsterixPropertiesAccessor createAsterixPropertiesAccessor() {
+ AsterixPropertiesAccessor propertiesAccessor = null;
+ try {
+ propertiesAccessor = new AsterixPropertiesAccessor();
+ } catch (AsterixException e) {
+ throw new IllegalStateException("Unable to create properties accessor");
+ }
+ return propertiesAccessor;
}
public void initialize() throws IOException, ACIDException, AsterixException {
- AsterixPropertiesAccessor propertiesAccessor = new AsterixPropertiesAccessor();
- compilerProperties = new AsterixCompilerProperties(propertiesAccessor);
- externalProperties = new AsterixExternalProperties(propertiesAccessor);
- metadataProperties = new AsterixMetadataProperties(propertiesAccessor);
- storageProperties = new AsterixStorageProperties(propertiesAccessor);
- txnProperties = new AsterixTransactionProperties(propertiesAccessor);
-
Logger.getLogger("edu.uci.ics").setLevel(externalProperties.getLogLevel());
fileMapManager = new AsterixFileMapManager();
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/DDLAPIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/DDLAPIServlet.java
index 1ae4ba6..51af387 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/DDLAPIServlet.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/DDLAPIServlet.java
@@ -32,7 +32,8 @@
protected List<Statement.Kind> getAllowedStatements() {
Kind[] statementsArray = { Kind.DATAVERSE_DECL, Kind.DATAVERSE_DROP, Kind.DATASET_DECL, Kind.NODEGROUP_DECL,
Kind.NODEGROUP_DROP, Kind.TYPE_DECL, Kind.TYPE_DROP, Kind.CREATE_INDEX, Kind.INDEX_DECL,
- Kind.CREATE_DATAVERSE, Kind.DATASET_DROP, Kind.INDEX_DROP, Kind.CREATE_FUNCTION, Kind.FUNCTION_DROP };
+ Kind.CREATE_DATAVERSE, Kind.DATASET_DROP, Kind.INDEX_DROP, Kind.CREATE_FUNCTION, Kind.FUNCTION_DROP,
+ Kind.CREATE_FEED };
return Arrays.asList(statementsArray);
}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedDashboardServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedDashboardServlet.java
new file mode 100644
index 0000000..f044ac5
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedDashboardServlet.java
@@ -0,0 +1,133 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.api.http.servlet;
+
+import java.awt.image.BufferedImage;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import javax.imageio.ImageIO;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityDetails;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
+
+public class FeedDashboardServlet extends HttpServlet {
+ private static final long serialVersionUID = 1L;
+
+ private static final Logger LOGGER = Logger.getLogger(FeedDashboardServlet.class.getName());
+
+ @Override
+ public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
+ String resourcePath = null;
+ String requestURI = request.getRequestURI();
+
+ if (requestURI.equals("/")) {
+ response.setContentType("text/html");
+ resourcePath = "/feed/dashboard.html";
+ } else {
+ resourcePath = requestURI + ".html";
+ }
+
+ try {
+ InputStream is = FeedDashboardServlet.class.getResourceAsStream(resourcePath);
+ if (is == null) {
+ response.sendError(HttpServletResponse.SC_NOT_FOUND);
+ return;
+ }
+
+ // Special handler for font files and .png resources
+ if (resourcePath.endsWith(".png")) {
+
+ BufferedImage img = ImageIO.read(is);
+ OutputStream outputStream = response.getOutputStream();
+ String formatName = "png";
+ response.setContentType("image/png");
+ ImageIO.write(img, formatName, outputStream);
+ outputStream.close();
+ return;
+
+ }
+
+ response.setCharacterEncoding("utf-8");
+ InputStreamReader isr = new InputStreamReader(is);
+ StringBuilder sb = new StringBuilder();
+ BufferedReader br = new BufferedReader(isr);
+ String line = br.readLine();
+
+ while (line != null) {
+ sb.append(line + "\n");
+ line = br.readLine();
+ }
+
+ String feedName = request.getParameter("feed");
+ String datasetName = request.getParameter("dataset");
+ String dataverseName = request.getParameter("dataverse");
+
+ FeedConnectionId feedId = new FeedConnectionId(dataverseName, feedName, datasetName);
+
+ String outStr = null;
+ if (requestURI.startsWith("/webui/static")) {
+ outStr = sb.toString();
+ } else {
+ MetadataManager.INSTANCE.init();
+ MetadataTransactionContext ctx = MetadataManager.INSTANCE.beginTransaction();
+ FeedActivity activity = MetadataManager.INSTANCE.getRecentActivityOnFeedConnection(ctx, feedId,
+ FeedActivityType.FEED_BEGIN);
+ MetadataManager.INSTANCE.commitTransaction(ctx);
+
+ Map<String, String> activityDetails = activity.getFeedActivityDetails();
+
+ String host = activityDetails.get(FeedActivity.FeedActivityDetails.SUPER_FEED_MANAGER_HOST);
+ int port = Integer.parseInt(activityDetails
+ .get(FeedActivity.FeedActivityDetails.SUPER_FEED_MANAGER_PORT));
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(" Super Feed Maanger address :" + host + "[" + port + "]");
+ }
+
+ String ingestLocations = activityDetails.get(FeedActivityDetails.INGEST_LOCATIONS);
+ String computeLocations = activityDetails.get(FeedActivityDetails.COMPUTE_LOCATIONS);
+ String storageLocations = activityDetails.get(FeedActivityDetails.STORAGE_LOCATIONS);
+ String ingestionPolicy = activityDetails.get(FeedActivityDetails.FEED_POLICY_NAME);
+ String activeSince = activity.getLastUpdatedTimestamp();
+
+ outStr = String.format(sb.toString(), dataverseName, datasetName, feedName, ingestLocations,
+ computeLocations, storageLocations, ingestionPolicy, activeSince);
+ FeedServletUtil.initiateSubscription(feedId, host, port);
+ }
+
+ PrintWriter out = response.getWriter();
+ out.println(outStr);
+ } catch (ACIDException | MetadataException e) {
+ e.printStackTrace();
+ }
+ }
+
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedDataProviderServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedDataProviderServlet.java
new file mode 100644
index 0000000..6985019
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedDataProviderServlet.java
@@ -0,0 +1,119 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.api.http.servlet;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.hyracks.bootstrap.FeedLifecycleListener;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityDetails;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
+
+public class FeedDataProviderServlet extends HttpServlet {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
+
+ String feedName = request.getParameter("feed");
+ String datasetName = request.getParameter("dataset");
+ String dataverseName = request.getParameter("dataverse");
+
+ String report = getFeedReport(feedName, datasetName, dataverseName);
+ System.out.println(" REPORT " + report);
+ long timestamp = System.currentTimeMillis();
+ JSONObject obj = null;
+ if (report != null) {
+ JSONArray array = new JSONArray();
+ try {
+ obj = new JSONObject();
+ obj.put("type", "report");
+ obj.put("time", timestamp);
+ obj.put("value", report);
+ } catch (JSONException jsoe) {
+ throw new IOException(jsoe);
+ }
+ } else {
+ obj = verifyIfFeedIsAlive(dataverseName, feedName, datasetName);
+ }
+
+ PrintWriter out = response.getWriter();
+ out.println(obj.toString());
+ }
+
+ private String getFeedReport(String feedName, String datasetName, String dataverseName) {
+ FeedConnectionId feedId = new FeedConnectionId(dataverseName, feedName, datasetName);
+ LinkedBlockingQueue<String> queue = FeedLifecycleListener.INSTANCE.getFeedReportQueue(feedId);
+ String report = null;
+ try {
+ report = queue.poll(25, TimeUnit.SECONDS);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ return report;
+ }
+
+ private JSONObject verifyIfFeedIsAlive(String dataverseName, String feedName, String datasetName) {
+ JSONObject obj = new JSONObject();
+ try {
+ MetadataTransactionContext ctx = MetadataManager.INSTANCE.beginTransaction();
+ List<FeedActivity> feedActivities = MetadataManager.INSTANCE
+ .getActiveFeeds(ctx, dataverseName, datasetName);
+ FeedConnectionId feedId = new FeedConnectionId(dataverseName, feedName, datasetName);
+ FeedActivity activity = MetadataManager.INSTANCE.getRecentActivityOnFeedConnection(ctx, feedId, null);
+ switch(activity.getActivityType()){
+ case FEED_BEGIN:
+ Map<String, String> activityDetails = activity.getFeedActivityDetails();
+ String ingestLocations = activityDetails.get(FeedActivityDetails.INGEST_LOCATIONS);
+ String computeLocations = activityDetails.get(FeedActivityDetails.COMPUTE_LOCATIONS);
+ String storageLocations = activityDetails.get(FeedActivityDetails.STORAGE_LOCATIONS);
+ obj.put("status", "active");
+ obj.put("type", "reload");
+ obj.put("ingestLocations", ingestLocations);
+ obj.put("computeLocations", computeLocations);
+ obj.put("storageLocations", storageLocations);
+ System.out.println(" RE LOADING " + " ingestion at " + ingestLocations + " compute at "
+ + computeLocations + " storage at " + storageLocations);
+ break;
+ case FEED_FAILURE:
+ obj.put("status", "failed");
+ break;
+ case FEED_END:
+ obj.put("status", "ended");
+ break;
+ }
+ } catch (Exception e) {
+ // ignore
+ }
+ return obj;
+
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedServlet.java
new file mode 100644
index 0000000..546ff4f
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedServlet.java
@@ -0,0 +1,123 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.api.http.servlet;
+
+import java.awt.image.BufferedImage;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.util.List;
+
+import javax.imageio.ImageIO;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityDetails;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
+
+public class FeedServlet extends HttpServlet {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
+ String resourcePath = null;
+ String requestURI = request.getRequestURI();
+
+ if (requestURI.equals("/")) {
+ response.setContentType("text/html");
+ resourcePath = "/feed/home.html";
+ } else {
+ resourcePath = requestURI;
+ }
+
+ try {
+ InputStream is = FeedServlet.class.getResourceAsStream(resourcePath);
+ if (is == null) {
+ response.sendError(HttpServletResponse.SC_NOT_FOUND);
+ return;
+ }
+
+ // Special handler for font files and .png resources
+ if (resourcePath.endsWith(".png")) {
+
+ BufferedImage img = ImageIO.read(is);
+ OutputStream outputStream = response.getOutputStream();
+ String formatName = "png";
+ response.setContentType("image/png");
+ ImageIO.write(img, formatName, outputStream);
+ outputStream.close();
+ return;
+
+ }
+
+ response.setCharacterEncoding("utf-8");
+ InputStreamReader isr = new InputStreamReader(is);
+ StringBuilder sb = new StringBuilder();
+ BufferedReader br = new BufferedReader(isr);
+ String line = br.readLine();
+
+ while (line != null) {
+ sb.append(line + "\n");
+ line = br.readLine();
+ }
+
+ String outStr = null;
+ if (requestURI.startsWith("/webui/static")) {
+ outStr = sb.toString();
+ } else {
+ MetadataManager.INSTANCE.init();
+ MetadataTransactionContext ctx = MetadataManager.INSTANCE.beginTransaction();
+ List<FeedActivity> lfa = MetadataManager.INSTANCE.getActiveFeeds(ctx, null, null);
+ StringBuilder ldStr = new StringBuilder();
+ ldStr.append("<br />");
+ ldStr.append("<br />");
+ if (lfa == null || lfa.isEmpty()) {
+ ldStr.append("Currently there are no active feeds in the Asterix");
+ } else {
+ ldStr.append("Active Feeds");
+ }
+ FeedConnectionId feedId = null;
+ for (FeedActivity feedActivity : lfa) {
+ feedId = new FeedConnectionId(feedActivity.getDataverseName(), feedActivity.getFeedName(),
+ feedActivity.getDatasetName());
+ ldStr.append("<br />");
+ ldStr.append("<br />");
+ ldStr.append("<a href=\"/feed/dashboard?dataverse=" + feedActivity.getDataverseName() + "&feed="
+ + feedActivity.getFeedName() + "&dataset=" + feedActivity.getDatasetName() + "\">" + feedId
+ + "</a>");
+ ldStr.append("<br />");
+ }
+
+ outStr = String.format(sb.toString(), ldStr.toString());
+ MetadataManager.INSTANCE.commitTransaction(ctx);
+
+ }
+
+ PrintWriter out = response.getWriter();
+ out.println(outStr);
+ } catch (ACIDException | MetadataException e) {
+
+ }
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedServletUtil.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedServletUtil.java
new file mode 100644
index 0000000..433f834
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedServletUtil.java
@@ -0,0 +1,45 @@
+package edu.uci.ics.asterix.api.http.servlet;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.Socket;
+import java.nio.CharBuffer;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.hyracks.bootstrap.FeedLifecycleListener;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.metadata.feeds.RemoteSocketMessageListener;
+
+public class FeedServletUtil {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedServletUtil.class.getName());
+ private static final char EOL = (char) "\n".getBytes()[0];
+
+ public static void initiateSubscription(FeedConnectionId feedId, String host, int port) throws IOException {
+ LinkedBlockingQueue<String> outbox = new LinkedBlockingQueue<String>();
+ int subscriptionPort = port + 1;
+ Socket sc = new Socket(host, subscriptionPort);
+ InputStream in = sc.getInputStream();
+
+ CharBuffer buffer = CharBuffer.allocate(50);
+ char ch = 0;
+ while (ch != EOL) {
+ buffer.put(ch);
+ ch = (char) in.read();
+ }
+ buffer.flip();
+ String s = new String(buffer.array());
+ int feedSubscriptionPort = Integer.parseInt(s.trim());
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Response from Super Feed Manager Report Service " + port + " will connect at " + host + " "
+ + port);
+ }
+
+ // register the feed subscription queue with FeedLifecycleListener
+ FeedLifecycleListener.INSTANCE.registerFeedReportQueue(feedId, outbox);
+ RemoteSocketMessageListener listener = new RemoteSocketMessageListener(host, feedSubscriptionPort, outbox);
+ listener.start();
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/UpdateAPIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/UpdateAPIServlet.java
index 772fad6..ac91636 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/UpdateAPIServlet.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/UpdateAPIServlet.java
@@ -30,9 +30,8 @@
}
protected List<Statement.Kind> getAllowedStatements() {
- Kind[] statementsArray = { Kind.DATAVERSE_DECL, Kind.DELETE, Kind.INSERT, Kind.UPDATE,
- Kind.DML_CMD_LIST, Kind.LOAD_FROM_FILE, Kind.BEGIN_FEED,
- Kind.CONTROL_FEED, Kind.COMPACT };
+ Kind[] statementsArray = { Kind.DATAVERSE_DECL, Kind.DELETE, Kind.INSERT, Kind.UPDATE, Kind.DML_CMD_LIST,
+ Kind.LOAD_FROM_FILE, Kind.CONNECT_FEED, Kind.DISCONNECT_FEED, Kind.SET, Kind.COMPACT };
return Arrays.asList(statementsArray);
}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
index 929465a..c81808c 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
@@ -20,8 +20,13 @@
import java.rmi.RemoteException;
import java.util.ArrayList;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
import org.json.JSONArray;
import org.json.JSONException;
@@ -32,19 +37,20 @@
import edu.uci.ics.asterix.api.common.Job;
import edu.uci.ics.asterix.api.common.SessionConfig;
import edu.uci.ics.asterix.aql.base.Statement;
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.CompactStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
import edu.uci.ics.asterix.aql.expression.DatasetDecl;
import edu.uci.ics.asterix.aql.expression.DataverseDecl;
import edu.uci.ics.asterix.aql.expression.DataverseDropStatement;
import edu.uci.ics.asterix.aql.expression.DeleteStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.DropStatement;
import edu.uci.ics.asterix.aql.expression.ExternalDetailsDecl;
-import edu.uci.ics.asterix.aql.expression.FeedDetailsDecl;
+import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
import edu.uci.ics.asterix.aql.expression.FunctionDecl;
import edu.uci.ics.asterix.aql.expression.FunctionDropStatement;
import edu.uci.ics.asterix.aql.expression.Identifier;
@@ -76,30 +82,38 @@
import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
+import edu.uci.ics.asterix.metadata.dataset.hints.DatasetHints;
+import edu.uci.ics.asterix.metadata.dataset.hints.DatasetHints.DatasetNodegroupCardinalityHint;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.entities.CompactionPolicy;
import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
-import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.Feed;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
import edu.uci.ics.asterix.metadata.entities.Function;
import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
import edu.uci.ics.asterix.metadata.entities.NodeGroup;
+import edu.uci.ics.asterix.metadata.feeds.BuiltinFeedPolicies;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.metadata.feeds.FeedUtil;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.ATypeTag;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.om.types.TypeSignature;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
import edu.uci.ics.asterix.result.ResultReader;
import edu.uci.ics.asterix.result.ResultUtils;
import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetIdFactory;
import edu.uci.ics.asterix.translator.AbstractAqlTranslator;
-import edu.uci.ics.asterix.translator.CompiledStatements.CompiledBeginFeedStatement;
-import edu.uci.ics.asterix.translator.CompiledStatements.CompiledControlFeedStatement;
+import edu.uci.ics.asterix.translator.CompiledStatements.CompiledConnectFeedStatement;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledDatasetDropStatement;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledDeleteStatement;
+import edu.uci.ics.asterix.translator.CompiledStatements.CompiledDisconnectFeedStatement;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledIndexCompactStatement;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledIndexDropStatement;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledInsertStatement;
@@ -128,6 +142,8 @@
*/
public class AqlTranslator extends AbstractAqlTranslator {
+ private static Logger LOGGER = Logger.getLogger(AqlTranslator.class.getName());
+
private enum ProgressState {
NO_PROGRESS,
ADDED_PENDINGOP_RECORD_TO_METADATA
@@ -261,13 +277,22 @@
break;
}
- case BEGIN_FEED: {
- handleBeginFeedStatement(metadataProvider, stmt, hcc);
+ case CREATE_FEED: {
+ handleCreateFeedStatement(metadataProvider, stmt, hcc);
break;
}
- case CONTROL_FEED: {
- handleControlFeedStatement(metadataProvider, stmt, hcc);
+ case DROP_FEED: {
+ handleDropFeedStatement(metadataProvider, stmt, hcc);
+ break;
+ }
+ case CONNECT_FEED: {
+ handleConnectFeedStatement(metadataProvider, stmt, hcc);
+ break;
+ }
+
+ case DISCONNECT_FEED: {
+ handleDisconnectFeedStatement(metadataProvider, stmt, hcc);
break;
}
@@ -432,7 +457,11 @@
.getPartitioningExprs();
ARecordType aRecordType = (ARecordType) itemType;
aRecordType.validatePartitioningExpressions(partitioningExprs);
- String ngName = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName().getValue();
+
+ Identifier ngNameId = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName();
+ String ngName = ngNameId != null ? ngNameId.getValue() : configureNodegroupForDataset(dd,
+ dataverseName, mdTxnCtx);
+
String compactionPolicy = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getCompactionPolicy();
Map<String, String> compactionPolicyProperties = ((InternalDetailsDecl) dd.getDatasetDetailsDecl())
.getCompactionPolicyProperties();
@@ -453,37 +482,8 @@
datasetDetails = new ExternalDatasetDetails(adapter, properties);
break;
}
- case FEED: {
- IAType itemType = dt.getDatatype();
- if (itemType.getTypeTag() != ATypeTag.RECORD) {
- throw new AlgebricksException("Can only partition ARecord's.");
- }
- List<String> partitioningExprs = ((FeedDetailsDecl) dd.getDatasetDetailsDecl())
- .getPartitioningExprs();
- ARecordType aRecordType = (ARecordType) itemType;
- aRecordType.validatePartitioningExpressions(partitioningExprs);
- String ngName = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName().getValue();
- String adapter = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getAdapterFactoryClassname();
- Map<String, String> configuration = ((FeedDetailsDecl) dd.getDatasetDetailsDecl())
- .getConfiguration();
- FunctionSignature signature = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getFunctionSignature();
- String compactionPolicy = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getCompactionPolicy();
- Map<String, String> compactionPolicyProperties = ((FeedDetailsDecl) dd.getDatasetDetailsDecl())
- .getCompactionPolicyProperties();
- if (compactionPolicy == null) {
- compactionPolicy = GlobalConfig.DEFAULT_COMPACTION_POLICY_NAME;
- compactionPolicyProperties = GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES;
- } else {
- validateCompactionPolicy(compactionPolicy, compactionPolicyProperties, mdTxnCtx);
- }
- datasetDetails = new FeedDatasetDetails(InternalDatasetDetails.FileStructure.BTREE,
- InternalDatasetDetails.PartitioningStrategy.HASH, partitioningExprs, partitioningExprs,
- ngName, adapter, configuration, signature,
- FeedDatasetDetails.FeedState.INACTIVE.toString(), compactionPolicy,
- compactionPolicyProperties);
- break;
- }
- }
+
+ }
//#. initialize DatasetIdFactory if it is not initialized.
if (!DatasetIdFactory.isInitialized()) {
@@ -495,7 +495,7 @@
DatasetIdFactory.generateDatasetId(), IMetadataEntity.PENDING_ADD_OP);
MetadataManager.INSTANCE.addDataset(metadataProvider.getMetadataTxnContext(), dataset);
- if (dd.getDatasetType() == DatasetType.INTERNAL || dd.getDatasetType() == DatasetType.FEED) {
+ if (dd.getDatasetType() == DatasetType.INTERNAL) {
Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(),
dataverseName);
JobSpecification jobSpec = DatasetOperations.createDatasetJobSpec(dataverse, datasetName,
@@ -570,6 +570,58 @@
}
}
+ private String configureNodegroupForDataset(DatasetDecl dd, String dataverse, MetadataTransactionContext mdTxnCtx)
+ throws AsterixException {
+ int nodegroupCardinality = -1;
+ String nodegroupName;
+ String hintValue = dd.getHints().get(DatasetNodegroupCardinalityHint.NAME);
+ if (hintValue == null) {
+ nodegroupName = MetadataConstants.METADATA_DEFAULT_NODEGROUP_NAME;
+ return nodegroupName;
+ } else {
+ int numChosen = 0;
+ boolean valid = DatasetHints.validate(DatasetNodegroupCardinalityHint.NAME,
+ dd.getHints().get(DatasetNodegroupCardinalityHint.NAME)).first;
+ if (!valid) {
+ throw new AsterixException("Incorrect use of hint:" + DatasetNodegroupCardinalityHint.NAME);
+ } else {
+ nodegroupCardinality = Integer.parseInt(dd.getHints().get(DatasetNodegroupCardinalityHint.NAME));
+ }
+ Set<String> nodeNames = AsterixAppContextInfo.getInstance().getMetadataProperties().getNodeNames();
+ Set<String> nodeNamesClone = new HashSet<String>();
+ for (String node : nodeNames) {
+ nodeNamesClone.add(node);
+ }
+ String metadataNodeName = AsterixAppContextInfo.getInstance().getMetadataProperties().getMetadataNodeName();
+ List<String> selectedNodes = new ArrayList<String>();
+ selectedNodes.add(metadataNodeName);
+ numChosen++;
+ nodeNamesClone.remove(metadataNodeName);
+
+ if (numChosen < nodegroupCardinality) {
+ Random random = new Random();
+ String[] nodes = nodeNamesClone.toArray(new String[] {});
+ int[] b = new int[nodeNamesClone.size()];
+ for (int i = 0; i < b.length; i++) {
+ b[i] = i;
+ }
+
+ for (int i = 0; i < nodegroupCardinality - numChosen; i++) {
+ int selected = i + random.nextInt(nodeNamesClone.size() - i);
+ int selNodeIndex = b[selected];
+ selectedNodes.add(nodes[selNodeIndex]);
+ int temp = b[0];
+ b[0] = b[selected];
+ b[selected] = temp;
+ }
+ }
+ nodegroupName = dataverse + ":" + dd.getName().getValue();
+ MetadataManager.INSTANCE.addNodegroup(mdTxnCtx, new NodeGroup(nodegroupName, selectedNodes));
+ return nodegroupName;
+ }
+
+ }
+
private void handleCreateIndexStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
@@ -616,6 +668,19 @@
}
}
+ List<FeedActivity> feedActivities = MetadataManager.INSTANCE.getActiveFeeds(mdTxnCtx, dataverseName,
+ datasetName);
+ if (feedActivities != null && !feedActivities.isEmpty()) {
+ StringBuilder builder = new StringBuilder();
+
+ for (FeedActivity fa : feedActivities) {
+ builder.append(fa + "\n");
+ }
+ throw new AsterixException("Dataset" + datasetName
+ + " is currently being fed into by the following feeds " + "." + builder.toString()
+ + "\nOperation not supported.");
+ }
+
//#. add a new index with PendingAddOp
Index index = new Index(dataverseName, datasetName, indexName, stmtCreateIndex.getIndexType(),
stmtCreateIndex.getFieldExprs(), stmtCreateIndex.getGramLength(), false,
@@ -772,12 +837,22 @@
}
}
+ //# disconnect all feeds from any datasets in the dataverse.
+ List<FeedActivity> feedActivities = MetadataManager.INSTANCE.getActiveFeeds(mdTxnCtx, dataverseName, null);
+ DisconnectFeedStatement disStmt = null;
+ Identifier dvId = new Identifier(dataverseName);
+ for (FeedActivity fa : feedActivities) {
+ disStmt = new DisconnectFeedStatement(dvId, new Identifier(fa.getFeedName()), new Identifier(
+ fa.getDatasetName()));
+ handleDisconnectFeedStatement(metadataProvider, disStmt, hcc);
+ }
+
//#. prepare jobs which will drop corresponding datasets with indexes.
List<Dataset> datasets = MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx, dataverseName);
for (int j = 0; j < datasets.size(); j++) {
String datasetName = datasets.get(j).getDatasetName();
DatasetType dsType = datasets.get(j).getDatasetType();
- if (dsType == DatasetType.INTERNAL || dsType == DatasetType.FEED) {
+ if (dsType == DatasetType.INTERNAL) {
List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
datasetName);
@@ -890,7 +965,22 @@
}
}
- if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
+ List<FeedActivity> feedActivities = MetadataManager.INSTANCE.getActiveFeeds(mdTxnCtx, dataverseName,
+ datasetName);
+ List<JobSpecification> disconnectFeedJobSpecs = new ArrayList<JobSpecification>();
+ if (feedActivities != null && !feedActivities.isEmpty()) {
+ for (FeedActivity fa : feedActivities) {
+ JobSpecification jobSpec = FeedOperations.buildDisconnectFeedJobSpec(dataverseName,
+ fa.getFeedName(), datasetName, metadataProvider, fa);
+ disconnectFeedJobSpecs.add(jobSpec);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Disconnected feed " + fa.getFeedName() + " from dataset " + datasetName
+ + " as dataset is being dropped");
+ }
+ }
+ }
+
+ if (ds.getDatasetType() == DatasetType.INTERNAL) {
//#. prepare jobs to drop the datatset and the indexes in NC
List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
@@ -915,6 +1005,11 @@
bActiveTxn = false;
progress = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
+ //# disconnect the feeds
+ for (JobSpecification jobSpec : disconnectFeedJobSpecs) {
+ runJob(hcc, jobSpec, true);
+ }
+
//#. run the jobs
for (JobSpecification jobSpec : jobsToExecute) {
runJob(hcc, jobSpec, true);
@@ -927,6 +1022,13 @@
//#. finally, delete the dataset.
MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataverseName, datasetName);
+ // Drop the associated nodegroup
+ if (ds.getDatasetType() == DatasetType.INTERNAL) {
+ String nodegroup = ((InternalDatasetDetails) ds.getDatasetDetails()).getNodeGroupName();
+ if (!nodegroup.equalsIgnoreCase(MetadataConstants.METADATA_DEFAULT_NODEGROUP_NAME)) {
+ MetadataManager.INSTANCE.dropNodegroup(mdTxnCtx, dataverseName + ":" + datasetName);
+ }
+ }
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
} catch (Exception e) {
@@ -991,7 +1093,20 @@
+ dataverseName);
}
- if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
+ List<FeedActivity> feedActivities = MetadataManager.INSTANCE.getActiveFeeds(mdTxnCtx, dataverseName,
+ datasetName);
+ if (feedActivities != null && !feedActivities.isEmpty()) {
+ StringBuilder builder = new StringBuilder();
+
+ for (FeedActivity fa : feedActivities) {
+ builder.append(fa + "\n");
+ }
+ throw new AsterixException("Dataset" + datasetName
+ + " is currently being fed into by the following feeds " + "." + builder.toString()
+ + "\nOperation not supported.");
+ }
+
+ if (ds.getDatasetType() == DatasetType.INTERNAL) {
indexName = stmtIndexDrop.getIndexName().getValue();
Index index = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, indexName);
if (index == null) {
@@ -1307,57 +1422,183 @@
}
- private void handleBeginFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ private void handleCreateFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ IHyracksClientConnection hcc) throws Exception {
+
+ MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ acquireWriteLatch();
+
+ String dataverseName = null;
+ String feedName = null;
+ String adaptorName = null;
+ Feed feed = null;
+ try {
+ CreateFeedStatement cfs = (CreateFeedStatement) stmt;
+ dataverseName = getActiveDataverseName(cfs.getDataverseName());
+ feedName = cfs.getFeedName().getValue();
+ adaptorName = cfs.getAdaptorName();
+
+ feed = MetadataManager.INSTANCE.getFeed(metadataProvider.getMetadataTxnContext(), dataverseName, feedName);
+ if (feed != null) {
+ if (cfs.getIfNotExists()) {
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return;
+ } else {
+ throw new AlgebricksException("A feed with this name " + adaptorName + " already exists.");
+ }
+ }
+
+ feed = new Feed(dataverseName, feedName, adaptorName, cfs.getAdaptorConfiguration(),
+ cfs.getAppliedFunction());
+ MetadataManager.INSTANCE.addFeed(metadataProvider.getMetadataTxnContext(), feed);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (Exception e) {
+ abort(e, e, mdTxnCtx);
+ throw e;
+ } finally {
+ releaseWriteLatch();
+ }
+ }
+
+ private void handleDropFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ IHyracksClientConnection hcc) throws Exception {
+ MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ acquireWriteLatch();
+
+ try {
+ FeedDropStatement stmtFeedDrop = (FeedDropStatement) stmt;
+ String dataverseName = getActiveDataverseName(stmtFeedDrop.getDataverseName());
+ String feedName = stmtFeedDrop.getFeedName().getValue();
+ Feed feed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, dataverseName, feedName);
+ if (feed == null) {
+ if (!stmtFeedDrop.getIfExists()) {
+ throw new AlgebricksException("There is no feed with this name " + feedName + ".");
+ }
+ }
+
+ List<FeedActivity> feedActivities;
+ try {
+ feedActivities = MetadataManager.INSTANCE.getConnectFeedActivitiesForFeed(mdTxnCtx, dataverseName,
+ feedName);
+ MetadataManager.INSTANCE.dropFeed(mdTxnCtx, dataverseName, feedName);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (Exception e) {
+ MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+ throw new MetadataException(e);
+ }
+
+ List<JobSpecification> jobSpecs = new ArrayList<JobSpecification>();
+ for (FeedActivity feedActivity : feedActivities) {
+ JobSpecification jobSpec = FeedOperations.buildDisconnectFeedJobSpec(dataverseName, feedName,
+ feedActivity.getDatasetName(), metadataProvider, feedActivity);
+ jobSpecs.add(jobSpec);
+ }
+
+ for (JobSpecification spec : jobSpecs) {
+ runJob(hcc, spec, true);
+ }
+
+ } catch (Exception e) {
+ abort(e, e, mdTxnCtx);
+ throw e;
+ } finally {
+ releaseWriteLatch();
+ }
+ }
+
+ private void handleConnectFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
boolean bActiveTxn = true;
metadataProvider.setMetadataTxnContext(mdTxnCtx);
acquireReadLatch();
-
+ boolean readLatchAcquired = true;
try {
+ ConnectFeedStatement cfs = (ConnectFeedStatement) stmt;
+ String dataverseName = getActiveDataverseName(cfs.getDataverseName());
metadataProvider.setWriteTransaction(true);
- BeginFeedStatement bfs = (BeginFeedStatement) stmt;
- String dataverseName = getActiveDataverseName(bfs.getDataverseName());
- CompiledBeginFeedStatement cbfs = new CompiledBeginFeedStatement(dataverseName, bfs.getDatasetName()
- .getValue(), bfs.getQuery(), bfs.getVarCounter());
+ CompiledConnectFeedStatement cbfs = new CompiledConnectFeedStatement(dataverseName, cfs.getFeedName(), cfs
+ .getDatasetName().getValue(), cfs.getPolicy(), cfs.getQuery(), cfs.getVarCounter());
- Dataset dataset;
- dataset = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(), dataverseName, bfs
- .getDatasetName().getValue());
+ Dataset dataset = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(),
+ dataverseName, cfs.getDatasetName().getValue());
if (dataset == null) {
- throw new AsterixException("Unknown dataset :" + bfs.getDatasetName().getValue());
+ throw new AsterixException("Unknown target dataset :" + cfs.getDatasetName().getValue());
+ }
+
+ if (!dataset.getDatasetType().equals(DatasetType.INTERNAL)) {
+ throw new AsterixException("Statement not applicable. Dataset " + cfs.getDatasetName().getValue()
+ + " is not of required type " + DatasetType.INTERNAL);
+ }
+
+ Feed feed = MetadataManager.INSTANCE.getFeed(metadataProvider.getMetadataTxnContext(), dataverseName,
+ cfs.getFeedName());
+ if (feed == null) {
+ throw new AsterixException("Unknown source feed :" + cfs.getFeedName());
+ }
+
+ FeedConnectionId feedConnId = new FeedConnectionId(dataverseName, cfs.getFeedName(), cfs.getDatasetName()
+ .getValue());
+ FeedActivity recentActivity = MetadataManager.INSTANCE.getRecentActivityOnFeedConnection(mdTxnCtx,
+ feedConnId, null);
+ boolean isFeedActive = FeedUtil.isFeedActive(recentActivity);
+ if (isFeedActive && !cfs.forceConnect()) {
+ throw new AsterixException("Feed " + cfs.getDatasetName().getValue()
+ + " is currently ACTIVE. Operation not supported");
}
IDatasetDetails datasetDetails = dataset.getDatasetDetails();
- if (datasetDetails.getDatasetType() != DatasetType.FEED) {
- throw new IllegalArgumentException("Dataset " + bfs.getDatasetName().getValue()
- + " is not a feed dataset");
+ if (datasetDetails.getDatasetType() != DatasetType.INTERNAL) {
+ throw new AsterixException("Dataset " + cfs.getDatasetName().getValue() + " is not an interal dataset");
}
- bfs.initialize(metadataProvider.getMetadataTxnContext(), dataset);
- cbfs.setQuery(bfs.getQuery());
- metadataProvider.getConfig().put(FunctionUtils.IMPORT_PRIVATE_FUNCTIONS, "" + Boolean.TRUE);
- JobSpecification compiled = rewriteCompileQuery(metadataProvider, bfs.getQuery(), cbfs);
+ FeedPolicy feedPolicy = MetadataManager.INSTANCE.getFeedPolicy(mdTxnCtx, dataverseName,
+ cbfs.getPolicyName());
+ if (feedPolicy == null) {
+ feedPolicy = MetadataManager.INSTANCE.getFeedPolicy(mdTxnCtx,
+ MetadataConstants.METADATA_DATAVERSE_NAME, cbfs.getPolicyName());
+ if (feedPolicy == null) {
+ throw new AsterixException("Unknown feed policy" + cbfs.getPolicyName());
+ }
+ }
+
+ cfs.initialize(metadataProvider.getMetadataTxnContext(), dataset, feed);
+ cbfs.setQuery(cfs.getQuery());
+ metadataProvider.getConfig().put(FunctionUtils.IMPORT_PRIVATE_FUNCTIONS, "" + Boolean.TRUE);
+ metadataProvider.getConfig().put(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY, cbfs.getPolicyName());
+ JobSpecification compiled = rewriteCompileQuery(metadataProvider, cfs.getQuery(), cbfs);
+ JobSpecification newJobSpec = FeedUtil.alterJobSpecificationForFeed(compiled, feedConnId, feedPolicy);
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Altered feed ingestion spec to wrap operators");
+ }
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
-
- if (compiled != null) {
- runJob(hcc, compiled, true);
+ String waitForCompletionParam = metadataProvider.getConfig().get(ConnectFeedStatement.WAIT_FOR_COMPLETION);
+ boolean waitForCompletion = waitForCompletionParam == null ? false : Boolean
+ .valueOf(waitForCompletionParam);
+ if (waitForCompletion) {
+ releaseReadLatch();
+ readLatchAcquired = false;
}
-
+ runJob(hcc, newJobSpec, waitForCompletion);
} catch (Exception e) {
if (bActiveTxn) {
abort(e, e, mdTxnCtx);
}
throw e;
} finally {
- releaseReadLatch();
+ if (readLatchAcquired) {
+ releaseReadLatch();
+ }
}
}
- private void handleControlFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ private void handleDisconnectFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
boolean bActiveTxn = true;
@@ -1365,17 +1606,45 @@
acquireReadLatch();
try {
- ControlFeedStatement cfs = (ControlFeedStatement) stmt;
+ DisconnectFeedStatement cfs = (DisconnectFeedStatement) stmt;
String dataverseName = getActiveDataverseName(cfs.getDataverseName());
- CompiledControlFeedStatement clcfs = new CompiledControlFeedStatement(cfs.getOperationType(),
- dataverseName, cfs.getDatasetName().getValue(), cfs.getAlterAdapterConfParams());
- JobSpecification jobSpec = FeedOperations.buildControlFeedJobSpec(clcfs, metadataProvider);
+
+ String datasetName = cfs.getDatasetName().getValue();
+ Dataset dataset = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(),
+ dataverseName, cfs.getDatasetName().getValue());
+ if (dataset == null) {
+ throw new AsterixException("Unknown dataset :" + cfs.getDatasetName().getValue() + " in dataverse "
+ + dataverseName);
+ }
+ if (!dataset.getDatasetType().equals(DatasetType.INTERNAL)) {
+ throw new AsterixException("Statement not applicable. Dataset " + cfs.getDatasetName().getValue()
+ + " is not of required type " + DatasetType.INTERNAL);
+ }
+
+ Feed feed = MetadataManager.INSTANCE.getFeed(metadataProvider.getMetadataTxnContext(), dataverseName, cfs
+ .getFeedName().getValue());
+ if (feed == null) {
+ throw new AsterixException("Unknown source feed :" + cfs.getFeedName());
+ }
+
+ FeedActivity feedActivity = MetadataManager.INSTANCE.getRecentActivityOnFeedConnection(mdTxnCtx,
+ new FeedConnectionId(dataverseName, feed.getFeedName(), datasetName), null);
+
+ boolean isFeedActive = FeedUtil.isFeedActive(feedActivity);
+ if (!isFeedActive) {
+ throw new AsterixException("Feed " + cfs.getDatasetName().getValue()
+ + " is currently INACTIVE. Operation not supported");
+ }
+
+ CompiledDisconnectFeedStatement clcfs = new CompiledDisconnectFeedStatement(dataverseName, cfs
+ .getFeedName().getValue(), cfs.getDatasetName().getValue());
+
+ JobSpecification jobSpec = FeedOperations.buildDisconnectFeedJobSpec(dataverseName, cfs.getFeedName()
+ .getValue(), cfs.getDatasetName().getValue(), metadataProvider, feedActivity);
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
-
runJob(hcc, jobSpec, true);
-
} catch (Exception e) {
if (bActiveTxn) {
abort(e, e, mdTxnCtx);
@@ -1405,7 +1674,7 @@
if (ds == null) {
throw new AlgebricksException("There is no dataset with this name " + datasetName + " in dataverse "
+ dataverseName + ".");
- } else if (ds.getDatasetType() != DatasetType.INTERNAL && ds.getDatasetType() != DatasetType.FEED) {
+ } else if (ds.getDatasetType() != DatasetType.INTERNAL) {
throw new AlgebricksException("Cannot compact the extrenal dataset " + datasetName + ".");
}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
index da0bebc..9a48c21 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
@@ -207,7 +207,7 @@
if (dataset == null) {
throw new AsterixException("Could not find dataset " + datasetName + " in dataverse " + dataverseName);
}
- if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
+ if (dataset.getDatasetType() != DatasetType.INTERNAL) {
throw new AsterixException("Cannot load data into dataset (" + datasetName + ")" + "of type "
+ dataset.getDatasetType());
}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java
index c7ad8bd..5e52f3e 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java
@@ -14,21 +14,11 @@
*/
package edu.uci.ics.asterix.file;
-import java.io.File;
-import java.util.ArrayList;
-import java.util.List;
import java.util.logging.Logger;
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.external.feed.lifecycle.AlterFeedMessage;
-import edu.uci.ics.asterix.external.feed.lifecycle.FeedMessage;
-import edu.uci.ics.asterix.external.feed.lifecycle.IFeedMessage;
-import edu.uci.ics.asterix.external.feed.lifecycle.IFeedMessage.MessageType;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
-import edu.uci.ics.asterix.metadata.entities.Dataset;
-import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
-import edu.uci.ics.asterix.translator.CompiledStatements.CompiledControlFeedStatement;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -57,60 +47,17 @@
* @throws AsterixException
* @throws AlgebricksException
*/
- public static JobSpecification buildControlFeedJobSpec(CompiledControlFeedStatement controlFeedStatement,
- AqlMetadataProvider metadataProvider) throws AsterixException, AlgebricksException {
- switch (controlFeedStatement.getOperationType()) {
- case ALTER:
- case END: {
- return createSendMessageToFeedJobSpec(controlFeedStatement, metadataProvider);
- }
- default: {
- throw new AsterixException("Unknown Operation Type: " + controlFeedStatement.getOperationType());
- }
-
- }
- }
-
- private static JobSpecification createSendMessageToFeedJobSpec(CompiledControlFeedStatement controlFeedStatement,
- AqlMetadataProvider metadataProvider) throws AsterixException {
- String dataverseName = controlFeedStatement.getDataverseName() == null ? metadataProvider
- .getDefaultDataverseName() : controlFeedStatement.getDataverseName();
- String datasetName = controlFeedStatement.getDatasetName();
- String datasetPath = dataverseName + File.separator + datasetName;
-
- LOGGER.info(" DATASETPATH: " + datasetPath);
-
- Dataset dataset;
- try {
- dataset = metadataProvider.findDataset(dataverseName, datasetName);
- } catch (AlgebricksException e) {
- throw new AsterixException(e);
- }
- if (dataset == null) {
- throw new AsterixException("FEED DATASET: No metadata for dataset " + datasetName);
- }
- if (dataset.getDatasetType() != DatasetType.FEED) {
- throw new AsterixException("Operation not support for dataset type " + dataset.getDatasetType());
- }
+ public static JobSpecification buildDisconnectFeedJobSpec(String dataverseName, String feedName,
+ String datasetName, AqlMetadataProvider metadataProvider, FeedActivity feedActivity)
+ throws AsterixException, AlgebricksException {
JobSpecification spec = JobSpecificationUtils.createJobSpecification();
IOperatorDescriptor feedMessenger;
AlgebricksPartitionConstraint messengerPc;
- List<IFeedMessage> feedMessages = new ArrayList<IFeedMessage>();
- switch (controlFeedStatement.getOperationType()) {
- case END:
- feedMessages.add(new FeedMessage(MessageType.STOP));
- break;
- case ALTER:
- feedMessages.add(new AlterFeedMessage(controlFeedStatement.getProperties()));
- break;
- }
-
try {
- Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = metadataProvider.buildFeedMessengerRuntime(
- metadataProvider, spec, (FeedDatasetDetails) dataset.getDatasetDetails(), dataverseName,
- datasetName, feedMessages);
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = metadataProvider
+ .buildDisconnectFeedMessengerRuntime(spec, dataverseName, feedName, datasetName, feedActivity);
feedMessenger = p.first;
messengerPc = p.second;
} catch (AlgebricksException e) {
@@ -121,9 +68,7 @@
NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(spec);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, nullSink, messengerPc);
-
spec.connect(new OneToOneConnectorDescriptor(spec), feedMessenger, 0, nullSink, 0);
-
spec.addRoot(nullSink);
return spec;
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
index dc201dd..c2f9192 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
@@ -23,6 +23,9 @@
import edu.uci.ics.asterix.api.http.servlet.APIServlet;
import edu.uci.ics.asterix.api.http.servlet.DDLAPIServlet;
+import edu.uci.ics.asterix.api.http.servlet.FeedDashboardServlet;
+import edu.uci.ics.asterix.api.http.servlet.FeedDataProviderServlet;
+import edu.uci.ics.asterix.api.http.servlet.FeedServlet;
import edu.uci.ics.asterix.api.http.servlet.QueryAPIServlet;
import edu.uci.ics.asterix.api.http.servlet.QueryResultAPIServlet;
import edu.uci.ics.asterix.api.http.servlet.QueryStatusAPIServlet;
@@ -46,6 +49,8 @@
private Server webServer;
private Server jsonAPIServer;
+ private Server feedServer;
+
private static IAsterixStateProxy proxy;
private ICCApplicationContext appCtx;
@@ -56,19 +61,30 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Starting Asterix cluster controller");
}
+
appCtx.setThreadFactory(AsterixThreadFactory.INSTANCE);
- AsterixAppContextInfo.initialize(appCtx);
+ AsterixAppContextInfo.initialize(appCtx, getNewHyracksClientConnection());
+
proxy = AsterixStateProxy.registerRemoteObject();
appCtx.setDistributedState(proxy);
AsterixMetadataProperties metadataProperties = AsterixAppContextInfo.getInstance().getMetadataProperties();
MetadataManager.INSTANCE = new MetadataManager(proxy, metadataProperties);
+ AsterixAppContextInfo.getInstance().getCCApplicationContext()
+ .addJobLifecycleListener(FeedLifecycleListener.INSTANCE);
+
AsterixExternalProperties externalProperties = AsterixAppContextInfo.getInstance().getExternalProperties();
setupWebServer(externalProperties);
webServer.start();
+
setupJSONAPIServer(externalProperties);
jsonAPIServer.start();
+ ExternalLibraryBootstrap.setUpExternaLibraries(false);
+
+ setupFeedServer(externalProperties);
+ feedServer.start();
+
ccAppCtx.addClusterLifecycleListener(ClusterLifecycleListener.INSTANCE);
}
@@ -119,4 +135,21 @@
context.addServlet(new ServletHolder(new UpdateAPIServlet()), "/update");
context.addServlet(new ServletHolder(new DDLAPIServlet()), "/ddl");
}
+
+ private void setupFeedServer(AsterixExternalProperties externalProperties) throws Exception {
+ feedServer = new Server(externalProperties.getFeedServerPort());
+
+ ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
+ context.setContextPath("/");
+
+ IHyracksClientConnection hcc = getNewHyracksClientConnection();
+ context.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
+
+ feedServer.setHandler(context);
+ context.addServlet(new ServletHolder(new FeedServlet()), "/");
+ context.addServlet(new ServletHolder(new FeedDashboardServlet()), "/feed/dashboard");
+ context.addServlet(new ServletHolder(new FeedDataProviderServlet()), "/feed/data");
+
+ // add paths here
+ }
}
\ No newline at end of file
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ClusterLifecycleListener.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
index a864e61..6461b0c 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
@@ -14,22 +14,53 @@
*/
package edu.uci.ics.asterix.hyracks.bootstrap;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
import java.util.Map;
import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
import java.util.logging.Level;
import java.util.logging.Logger;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
+import edu.uci.ics.asterix.metadata.api.IClusterEventsSubscriber;
+import edu.uci.ics.asterix.metadata.api.IClusterManagementWork;
+import edu.uci.ics.asterix.metadata.cluster.AddNodeWork;
+import edu.uci.ics.asterix.metadata.cluster.AddNodeWorkResponse;
+import edu.uci.ics.asterix.metadata.cluster.ClusterManager;
+import edu.uci.ics.asterix.metadata.cluster.IClusterManagementWorkResponse;
+import edu.uci.ics.asterix.metadata.cluster.IClusterManagementWorkResponse.Status;
+import edu.uci.ics.asterix.metadata.cluster.RemoveNodeWork;
+import edu.uci.ics.asterix.metadata.cluster.RemoveNodeWorkResponse;
import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
import edu.uci.ics.hyracks.api.application.IClusterLifecycleListener;
public class ClusterLifecycleListener implements IClusterLifecycleListener {
+ private static final Logger LOGGER = Logger.getLogger(ClusterLifecycleListener.class.getName());
+
+ private static final LinkedBlockingQueue<Set<IClusterManagementWork>> workRequestQueue = new LinkedBlockingQueue<Set<IClusterManagementWork>>();
+
+ private static ClusterWorkExecutor eventHandler = new ClusterWorkExecutor(workRequestQueue);
+
+ private static List<IClusterManagementWorkResponse> pendingWorkResponses = new ArrayList<IClusterManagementWorkResponse>();
+
public static ClusterLifecycleListener INSTANCE = new ClusterLifecycleListener();
private ClusterLifecycleListener() {
+ Thread t = new Thread(eventHandler);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Starting cluster event handler");
+ }
+ t.start();
}
- private static final Logger LOGGER = Logger.getLogger(ClusterLifecycleListener.class.getName());
+ public enum ClusterEventType {
+ NODE_JOIN,
+ NODE_FAILURE
+ }
@Override
public void notifyNodeJoin(String nodeId, Map<String, String> ncConfiguration) {
@@ -37,6 +68,21 @@
LOGGER.info("NC: " + nodeId + " joined");
}
AsterixClusterProperties.INSTANCE.addNCConfiguration(nodeId, ncConfiguration);
+ Set<String> nodeAddition = new HashSet<String>();
+ nodeAddition.add(nodeId);
+ updateProgress(ClusterEventType.NODE_JOIN, nodeAddition);
+ Set<IClusterEventsSubscriber> subscribers = ClusterManager.INSTANCE.getRegisteredClusterEventSubscribers();
+ Set<IClusterManagementWork> work = new HashSet<IClusterManagementWork>();
+ for (IClusterEventsSubscriber sub : subscribers) {
+ Set<IClusterManagementWork> workRequest = sub.notifyNodeJoin(nodeId);
+ if (workRequest != null && !workRequest.isEmpty()) {
+ work.addAll(workRequest);
+ }
+ }
+ if (!work.isEmpty()) {
+ executeWorkSet(work);
+ }
+
}
public void notifyNodeFailure(Set<String> deadNodeIds) {
@@ -46,7 +92,123 @@
}
AsterixClusterProperties.INSTANCE.removeNCConfiguration(deadNode);
}
-
+ updateProgress(ClusterEventType.NODE_FAILURE, deadNodeIds);
+ Set<IClusterEventsSubscriber> subscribers = ClusterManager.INSTANCE.getRegisteredClusterEventSubscribers();
+ Set<IClusterManagementWork> work = new HashSet<IClusterManagementWork>();
+ for (IClusterEventsSubscriber sub : subscribers) {
+ Set<IClusterManagementWork> workRequest = sub.notifyNodeFailure(deadNodeIds);
+ if (workRequest != null && !workRequest.isEmpty()) {
+ work.addAll(workRequest);
+ }
+ }
+ if (!work.isEmpty()) {
+ executeWorkSet(work);
+ }
}
+ private void submitWork(Set<IClusterManagementWork> work) {
+ try {
+ workRequestQueue.put(work);
+ } catch (InterruptedException e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Interrupted :" + e.getMessage());
+ }
+ }
+ }
+
+ private void updateProgress(ClusterEventType eventType, Set<String> nodeIds) {
+ List<IClusterManagementWorkResponse> completedResponses = new ArrayList<IClusterManagementWorkResponse>();
+ boolean isComplete = false;
+ for (IClusterManagementWorkResponse resp : pendingWorkResponses) {
+ switch (eventType) {
+ case NODE_FAILURE:
+ isComplete = ((RemoveNodeWorkResponse) resp).updateProgress(nodeIds);
+ if (isComplete) {
+ resp.setStatus(Status.SUCCESS);
+ resp.getWork().getSourceSubscriber().notifyRequestCompletion(resp);
+ completedResponses.add(resp);
+ }
+ break;
+
+ case NODE_JOIN:
+ isComplete = ((AddNodeWorkResponse) resp).updateProgress(nodeIds.iterator().next());
+ if (isComplete) {
+ resp.setStatus(Status.SUCCESS);
+ resp.getWork().getSourceSubscriber().notifyRequestCompletion(resp);
+ completedResponses.add(resp);
+ }
+ break;
+ }
+ }
+ pendingWorkResponses.removeAll(completedResponses);
+ }
+
+ private void executeWorkSet(Set<IClusterManagementWork> workSet) {
+ int nodesToAdd = 0;
+ Set<String> nodesToRemove = new HashSet<String>();
+ Set<AddNodeWork> nodeAdditionRequests = new HashSet<AddNodeWork>();
+ Set<IClusterManagementWork> nodeRemovalRequests = new HashSet<IClusterManagementWork>();
+ for (IClusterManagementWork w : workSet) {
+ switch (w.getClusterManagementWorkType()) {
+ case ADD_NODE:
+ if (nodesToAdd < ((AddNodeWork) w).getNumberOfNodes()) {
+ nodesToAdd = ((AddNodeWork) w).getNumberOfNodes();
+ }
+ nodeAdditionRequests.add((AddNodeWork) w);
+ break;
+ case REMOVE_NODE:
+ nodesToRemove.addAll(((RemoveNodeWork) w).getNodesToBeRemoved());
+ nodeRemovalRequests.add(w);
+ RemoveNodeWorkResponse response = new RemoveNodeWorkResponse((RemoveNodeWork) w, Status.IN_PROGRESS);
+ pendingWorkResponses.add(response);
+ break;
+ }
+ }
+
+ List<String> addedNodes = new ArrayList<String>();
+ String asterixInstanceName = AsterixClusterProperties.INSTANCE.getCluster().getInstanceName();
+ for (int i = 0; i < nodesToAdd; i++) {
+ Node node = AsterixClusterProperties.INSTANCE.getAvailableSubstitutionNode();
+ if (node != null) {
+ try {
+ ClusterManager.INSTANCE.addNode(node);
+ addedNodes.add(asterixInstanceName + "_" + node.getId());
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Added NC at:" + node.getId());
+ }
+ } catch (AsterixException e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to add NC at:" + node.getId());
+ }
+ e.printStackTrace();
+ }
+ } else {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to add NC: no more available nodes");
+ }
+
+ }
+ }
+
+ for (AddNodeWork w : nodeAdditionRequests) {
+ int n = w.getNumberOfNodes();
+ List<String> nodesToBeAddedForWork = new ArrayList<String>();
+ for (int i = 0; i < n && i < addedNodes.size(); i++) {
+ nodesToBeAddedForWork.add(addedNodes.get(i));
+ }
+ if (nodesToBeAddedForWork.isEmpty()) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Unable to satisfy request by " + w);
+ }
+ AddNodeWorkResponse response = new AddNodeWorkResponse(w, nodesToBeAddedForWork);
+ response.setStatus(Status.FAILURE);
+ w.getSourceSubscriber().notifyRequestCompletion(response);
+
+ } else {
+ AddNodeWorkResponse response = new AddNodeWorkResponse(w, nodesToBeAddedForWork);
+ pendingWorkResponses.add(response);
+ }
+ }
+
+ }
}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ClusterWorkExecutor.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ClusterWorkExecutor.java
new file mode 100644
index 0000000..a6c1f8b
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ClusterWorkExecutor.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.hyracks.bootstrap;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
+import edu.uci.ics.asterix.metadata.api.IClusterManagementWork;
+import edu.uci.ics.asterix.metadata.cluster.AddNodeWork;
+import edu.uci.ics.asterix.metadata.cluster.AddNodeWorkResponse;
+import edu.uci.ics.asterix.metadata.cluster.ClusterManager;
+import edu.uci.ics.asterix.metadata.cluster.RemoveNodeWork;
+import edu.uci.ics.asterix.metadata.cluster.IClusterManagementWorkResponse.Status;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
+
+public class ClusterWorkExecutor implements Runnable {
+
+ private static final Logger LOGGER = Logger.getLogger(ClusterWorkExecutor.class.getName());
+
+ private final LinkedBlockingQueue<Set<IClusterManagementWork>> inbox;
+
+ public ClusterWorkExecutor(LinkedBlockingQueue<Set<IClusterManagementWork>> inbox) {
+ this.inbox = inbox;
+ }
+
+
+ @Override
+ public void run() {
+ while (true) {
+ try {
+ Set<IClusterManagementWork> workSet = inbox.take();
+ int nodesToAdd = 0;
+ Set<String> nodesToRemove = new HashSet<String>();
+ Set<IClusterManagementWork> nodeAdditionRequests = new HashSet<IClusterManagementWork>();
+ Set<IClusterManagementWork> nodeRemovalRequests = new HashSet<IClusterManagementWork>();
+ for (IClusterManagementWork w : workSet) {
+ switch (w.getClusterManagementWorkType()) {
+ case ADD_NODE:
+ if (nodesToAdd < ((AddNodeWork) w).getNumberOfNodes()) {
+ nodesToAdd = ((AddNodeWork) w).getNumberOfNodes();
+ }
+ nodeAdditionRequests.add(w);
+ break;
+ case REMOVE_NODE:
+ nodesToRemove.addAll(((RemoveNodeWork) w).getNodesToBeRemoved());
+ nodeRemovalRequests.add(w);
+ break;
+ }
+ }
+
+ Set<Node> addedNodes = new HashSet<Node>();
+ for (int i = 0; i < nodesToAdd; i++) {
+ Node node = AsterixClusterProperties.INSTANCE.getAvailableSubstitutionNode();
+ if (node != null) {
+ try {
+ ClusterManager.INSTANCE.addNode(node);
+ addedNodes.add(node);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Added NC at:" + node.getId());
+ }
+ } catch (AsterixException e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to add NC at:" + node.getId());
+ }
+ e.printStackTrace();
+ }
+ } else {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to add NC: no more available nodes");
+ }
+ }
+ }
+
+
+ } catch (InterruptedException e) {
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("interruped" + e.getMessage());
+ }
+ throw new IllegalStateException(e);
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("Unexpected exception in handling cluster event" + e.getMessage());
+ }
+ }
+
+ }
+ }
+
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ExternalLibraryBootstrap.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ExternalLibraryBootstrap.java
new file mode 100755
index 0000000..ecf1a14e
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ExternalLibraryBootstrap.java
@@ -0,0 +1,300 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.hyracks.bootstrap;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.rmi.RemoteException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.Unmarshaller;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.asterix.external.library.ExternalLibraryManager;
+import edu.uci.ics.asterix.external.library.Function;
+import edu.uci.ics.asterix.external.library.Functions;
+import edu.uci.ics.asterix.external.library.Library;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
+import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
+
+public class ExternalLibraryBootstrap {
+
+ public static void setUpExternaLibraries(boolean isMetadataNode) throws Exception {
+
+ Map<String, List<String>> uninstalledLibs = null;
+ if (isMetadataNode) {
+ uninstalledLibs = uninstallLibraries();
+ }
+
+ File installLibDir = getLibraryInstallDir();
+ if (installLibDir.exists()) {
+ for (String dataverse : installLibDir.list()) {
+ File dataverseDir = new File(installLibDir, dataverse);
+ String[] libraries = dataverseDir.list();
+ for (String library : libraries) {
+ registerLibrary(dataverse, library, isMetadataNode, installLibDir);
+ if (isMetadataNode) {
+ File libraryDir = new File(installLibDir.getAbsolutePath() + File.separator + dataverse
+ + File.separator + library);
+ installLibraryIfNeeded(dataverse, libraryDir, uninstalledLibs);
+ }
+ }
+ }
+ }
+ }
+
+ private static Map<String, List<String>> uninstallLibraries() throws Exception {
+ Map<String, List<String>> uninstalledLibs = new HashMap<String, List<String>>();
+ File uninstallLibDir = getLibraryUninstallDir();
+ String[] uninstallLibNames;
+ if (uninstallLibDir.exists()) {
+ uninstallLibNames = uninstallLibDir.list();
+ for (String uninstallLibName : uninstallLibNames) {
+ String[] components = uninstallLibName.split("\\.");
+ String dataverse = components[0];
+ String libName = components[1];
+ uninstallLibrary(dataverse, libName);
+ new File(uninstallLibDir, uninstallLibName).delete();
+ List<String> uinstalledLibsInDv = uninstalledLibs.get(dataverse);
+ if (uinstalledLibsInDv == null) {
+ uinstalledLibsInDv = new ArrayList<String>();
+ uninstalledLibs.put(dataverse, uinstalledLibsInDv);
+ }
+ uinstalledLibsInDv.add(libName);
+ }
+ }
+ return uninstalledLibs;
+ }
+
+ private static boolean uninstallLibrary(String dataverse, String libraryName) throws AsterixException,
+ RemoteException, ACIDException {
+ MetadataTransactionContext mdTxnCtx = null;
+ try {
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverse);
+ if (dv == null) {
+ return false;
+ }
+
+ edu.uci.ics.asterix.metadata.entities.Library library = MetadataManager.INSTANCE.getLibrary(mdTxnCtx,
+ dataverse, libraryName);
+ if (library == null) {
+ return false;
+ }
+
+ List<edu.uci.ics.asterix.metadata.entities.Function> functions = MetadataManager.INSTANCE
+ .getDataverseFunctions(mdTxnCtx, dataverse);
+ for (edu.uci.ics.asterix.metadata.entities.Function function : functions) {
+ if (function.getName().startsWith(libraryName + ":")) {
+ MetadataManager.INSTANCE.dropFunction(mdTxnCtx, new FunctionSignature(dataverse,
+ function.getName(), function.getArity()));
+ }
+ }
+
+ MetadataManager.INSTANCE.dropLibrary(mdTxnCtx, dataverse, libraryName);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (Exception e) {
+ MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+ throw new AsterixException(e);
+ }
+ return true;
+ }
+
+ // Each element of a library is installed as part of a transaction. Any
+ // failure in installing an element does not effect installation of other
+ // libraries
+ private static void installLibraryIfNeeded(String dataverse, final File libraryDir,
+ Map<String, List<String>> uninstalledLibs) throws Exception {
+
+ String libraryName = libraryDir.getName();
+ List<String> uninstalledLibsInDv = uninstalledLibs.get(dataverse);
+ boolean wasUninstalled = uninstalledLibsInDv != null && uninstalledLibsInDv.contains(libraryName);
+
+ MetadataTransactionContext mdTxnCtx = null;
+ try {
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ edu.uci.ics.asterix.metadata.entities.Library libraryInMetadata = MetadataManager.INSTANCE.getLibrary(
+ mdTxnCtx, dataverse, libraryName);
+ if (libraryInMetadata != null && !wasUninstalled) {
+ return;
+ }
+
+ String[] libraryDescriptors = libraryDir.list(new FilenameFilter() {
+ @Override
+ public boolean accept(File dir, String name) {
+ return name.endsWith(".xml");
+ }
+ });
+
+ Library library = getLibrary(new File(libraryDir + File.separator + libraryDescriptors[0]));
+
+ if (libraryDescriptors.length == 0) {
+ throw new Exception("No library descriptors defined");
+ } else if (libraryDescriptors.length > 1) {
+ throw new Exception("More than 1 library descriptors defined");
+ }
+
+ Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverse);
+ if (dv == null) {
+ MetadataManager.INSTANCE.addDataverse(mdTxnCtx, new Dataverse(dataverse,
+ NonTaggedDataFormat.NON_TAGGED_DATA_FORMAT, IMetadataEntity.PENDING_NO_OP));
+ }
+ for (Function function : library.getFunctions().getFunction()) {
+ String[] fargs = function.getArguments().trim().split(",");
+ List<String> args = new ArrayList<String>();
+ for (String arg : fargs) {
+ args.add(arg);
+ }
+ edu.uci.ics.asterix.metadata.entities.Function f = new edu.uci.ics.asterix.metadata.entities.Function(
+ dataverse, libraryName + "#" + function.getName(), args.size(), args, function.getReturnType(),
+ function.getDefinition(), library.getLanguage(), function.getFunctionType());
+ MetadataManager.INSTANCE.addFunction(mdTxnCtx, f);
+ }
+
+ MetadataManager.INSTANCE.addLibrary(mdTxnCtx, new edu.uci.ics.asterix.metadata.entities.Library(dataverse,
+ libraryName));
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (Exception e) {
+ e.printStackTrace();
+ MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+ }
+ }
+
+ private static void registerLibrary(String dataverse, String libraryName, boolean isMetadataNode, File installLibDir)
+ throws Exception {
+ ClassLoader classLoader = getLibraryClassLoader(dataverse, libraryName);
+ ExternalLibraryManager.registerLibraryClassLoader(dataverse, libraryName, classLoader);
+ }
+
+ private static Library getLibrary(File libraryXMLPath) throws Exception {
+ JAXBContext configCtx = JAXBContext.newInstance(Library.class);
+ Unmarshaller unmarshaller = configCtx.createUnmarshaller();
+ Library library = (Library) unmarshaller.unmarshal(libraryXMLPath);
+ return library;
+ }
+
+ private static ClassLoader getLibraryClassLoader(String dataverse, String libraryName) throws Exception {
+ System.out.println(" installing lirbary " + libraryName + " in dataverse " + dataverse);
+ File installDir = getLibraryInstallDir();
+ System.out.println(" install directory " + installDir.getAbsolutePath());
+
+ File libDir = new File(installDir.getAbsolutePath() + File.separator + dataverse + File.separator + libraryName);
+ FilenameFilter jarFileFilter = new FilenameFilter() {
+ public boolean accept(File dir, String name) {
+ return name.endsWith(".jar");
+ }
+ };
+
+ String[] jarsInLibDir = libDir.list(jarFileFilter);
+ System.out.println(" jars in lib dir " + jarsInLibDir);
+
+ if (jarsInLibDir.length > 1) {
+ throw new Exception("Incorrect library structure: found multiple library jars");
+ }
+ if (jarsInLibDir.length < 0) {
+ throw new Exception("Incorrect library structure: could not find library jar");
+ }
+
+ File libJar = new File(libDir, jarsInLibDir[0]);
+ File libDependencyDir = new File(libDir.getAbsolutePath() + File.separator + "lib");
+ int numDependencies = 1;
+ String[] libraryDependencies = null;
+ if (libDependencyDir.exists()) {
+ libraryDependencies = libDependencyDir.list(jarFileFilter);
+ numDependencies += libraryDependencies.length;
+ }
+
+ ClassLoader parentClassLoader = ExternalLibraryBootstrap.class.getClassLoader();
+ URL[] urls = new URL[numDependencies];
+ int count = 0;
+ urls[count++] = libJar.toURL();
+
+ if (libraryDependencies != null && libraryDependencies.length > 0) {
+ for (String dependency : libraryDependencies) {
+ File file = new File(libDependencyDir + File.separator + dependency);
+ urls[count++] = file.toURL();
+ }
+ }
+ ClassLoader classLoader = new URLClassLoader(urls, parentClassLoader);
+ return classLoader;
+ }
+
+ private static File getLibraryInstallDir() {
+ String workingDir = System.getProperty("user.dir");
+ return new File(workingDir + File.separator + "library");
+ }
+
+ private static File getLibraryUninstallDir() {
+ String workingDir = System.getProperty("user.dir");
+ return new File(workingDir + File.separator + "uninstall");
+ }
+
+}
+
+class ExternalLibrary {
+
+ private final String dataverse;
+ private final String name;
+ private final String language;
+ private final Functions functions;
+
+ public ExternalLibrary(String dataverse, String name, String language, Functions functions) {
+ this.dataverse = dataverse;
+ this.name = name;
+ this.language = language;
+ this.functions = functions;
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder("");
+ builder.append("Library");
+ builder.append("\n");
+ builder.append("Functions");
+ builder.append("\n");
+ for (Function function : functions.getFunction()) {
+ builder.append(function);
+ builder.append("\n");
+ }
+ return new String(builder);
+ }
+
+ public String getDataverse() {
+ return dataverse;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public String getLanguage() {
+ return language;
+ }
+
+ public Functions getFunctions() {
+ return functions;
+ }
+
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedLifecycleListener.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedLifecycleListener.java
new file mode 100644
index 0000000..37c6e8a
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedLifecycleListener.java
@@ -0,0 +1,1172 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.hyracks.bootstrap;
+
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.rmi.RemoteException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.api.common.APIFramework.DisplayFormat;
+import edu.uci.ics.asterix.api.common.SessionConfig;
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.DataverseDecl;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.Identifier;
+import edu.uci.ics.asterix.aql.translator.AqlTranslator;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.event.schema.cluster.Cluster;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
+import edu.uci.ics.asterix.file.JobSpecificationUtils;
+import edu.uci.ics.asterix.hyracks.bootstrap.FeedLifecycleListener.FeedFailure.FailureType;
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.api.IClusterEventsSubscriber;
+import edu.uci.ics.asterix.metadata.api.IClusterManagementWork;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
+import edu.uci.ics.asterix.metadata.cluster.AddNodeWork;
+import edu.uci.ics.asterix.metadata.cluster.ClusterManager;
+import edu.uci.ics.asterix.metadata.cluster.IClusterManagementWorkResponse;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityDetails;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
+import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
+import edu.uci.ics.asterix.metadata.feeds.BuiltinFeedPolicies;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.metadata.feeds.FeedIntakeOperatorDescriptor;
+import edu.uci.ics.asterix.metadata.feeds.FeedManagerElectMessage;
+import edu.uci.ics.asterix.metadata.feeds.FeedMetaOperatorDescriptor;
+import edu.uci.ics.asterix.metadata.feeds.FeedPolicyAccessor;
+import edu.uci.ics.asterix.metadata.feeds.IFeedMessage;
+import edu.uci.ics.asterix.metadata.feeds.MessageListener;
+import edu.uci.ics.asterix.metadata.feeds.MessageListener.IMessageAnalyzer;
+import edu.uci.ics.asterix.metadata.feeds.SuperFeedManager;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties.State;
+import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.constraints.Constraint;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.LValueConstraintExpression;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
+import edu.uci.ics.hyracks.api.job.IJobLifecycleListener;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobInfo;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.api.job.JobStatus;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexInsertUpdateDeleteOperatorDescriptor;
+
+public class FeedLifecycleListener implements IJobLifecycleListener, IClusterEventsSubscriber, Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ private static final Logger LOGGER = Logger.getLogger(FeedLifecycleListener.class.getName());
+
+ public static FeedLifecycleListener INSTANCE = new FeedLifecycleListener();
+
+ public static final int FEED_HEALTH_PORT = 2999;
+
+ private LinkedBlockingQueue<Message> jobEventInbox;
+ private LinkedBlockingQueue<IClusterManagementWorkResponse> responseInbox;
+ private Map<FeedInfo, List<String>> dependentFeeds = new HashMap<FeedInfo, List<String>>();
+ private IMessageAnalyzer healthDataParser;
+ private MessageListener feedHealthDataListener;
+ private ExecutorService executorService = Executors.newCachedThreadPool();
+ private Map<FeedConnectionId, LinkedBlockingQueue<String>> feedReportQueue = new HashMap<FeedConnectionId, LinkedBlockingQueue<String>>();
+ private State state;
+
+ private FeedLifecycleListener() {
+ jobEventInbox = new LinkedBlockingQueue<Message>();
+ feedJobNotificationHandler = new FeedJobNotificationHandler(jobEventInbox);
+ responseInbox = new LinkedBlockingQueue<IClusterManagementWorkResponse>();
+ feedWorkRequestResponseHandler = new FeedWorkRequestResponseHandler(responseInbox);
+ this.healthDataParser = new FeedHealthDataParser();
+ feedHealthDataListener = new MessageListener(FEED_HEALTH_PORT, healthDataParser.getMessageQueue());
+ try {
+ feedHealthDataListener.start();
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to start Feed health data listener");
+ }
+ }
+ executorService.execute(feedJobNotificationHandler);
+ executorService.execute(feedWorkRequestResponseHandler);
+ ClusterManager.INSTANCE.registerSubscriber(this);
+ state = AsterixClusterProperties.INSTANCE.getState();
+
+ }
+
+ private final FeedJobNotificationHandler feedJobNotificationHandler;
+ private final FeedWorkRequestResponseHandler feedWorkRequestResponseHandler;
+
+ @Override
+ public void notifyJobStart(JobId jobId) throws HyracksException {
+ if (feedJobNotificationHandler.isRegisteredFeed(jobId)) {
+ jobEventInbox.add(new Message(jobId, Message.MessageKind.JOB_START));
+ }
+ }
+
+ @Override
+ public void notifyJobFinish(JobId jobId) throws HyracksException {
+ if (feedJobNotificationHandler.isRegisteredFeed(jobId)) {
+ jobEventInbox.add(new Message(jobId, Message.MessageKind.JOB_FINISH));
+ }
+ }
+
+ @Override
+ public void notifyJobCreation(JobId jobId, IActivityClusterGraphGeneratorFactory acggf) throws HyracksException {
+
+ JobSpecification spec = acggf.getJobSpecification();
+ boolean feedIngestionJob = false;
+ FeedConnectionId feedId = null;
+ Map<String, String> feedPolicy = null;
+ for (IOperatorDescriptor opDesc : spec.getOperatorMap().values()) {
+ if (!(opDesc instanceof FeedIntakeOperatorDescriptor)) {
+ continue;
+ }
+ feedId = ((FeedIntakeOperatorDescriptor) opDesc).getFeedId();
+ feedPolicy = ((FeedIntakeOperatorDescriptor) opDesc).getFeedPolicy();
+ feedIngestionJob = true;
+ break;
+ }
+ if (feedIngestionJob) {
+ feedJobNotificationHandler.registerFeed(feedId, jobId, spec, feedPolicy);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Registered feed: " + feedId + " ingestion policy "
+ + feedPolicy.get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY));
+ }
+ }
+
+ }
+
+ public void registerFeedReportQueue(FeedConnectionId feedId, LinkedBlockingQueue<String> queue) {
+ feedReportQueue.put(feedId, queue);
+ }
+
+ public void deregisterFeedReportQueue(FeedConnectionId feedId, LinkedBlockingQueue<String> queue) {
+ feedReportQueue.remove(feedId);
+ }
+
+ public LinkedBlockingQueue<String> getFeedReportQueue(FeedConnectionId feedId) {
+ return feedReportQueue.get(feedId);
+ }
+
+ private static class Message {
+ public JobId jobId;
+
+ public enum MessageKind {
+ JOB_START,
+ JOB_FINISH
+ }
+
+ public MessageKind messageKind;
+
+ public Message(JobId jobId, MessageKind msgKind) {
+ this.jobId = jobId;
+ this.messageKind = msgKind;
+ }
+ }
+
+ public static class FeedFailureReport {
+ public Map<FeedInfo, List<FeedFailure>> failures = new HashMap<FeedInfo, List<FeedFailure>>();
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ for (Map.Entry<FeedLifecycleListener.FeedInfo, List<FeedLifecycleListener.FeedFailure>> entry : failures
+ .entrySet()) {
+ builder.append(entry.getKey() + " -> failures");
+ for (FeedFailure failure : entry.getValue()) {
+ builder.append("failure -> " + failure);
+ }
+ }
+ return builder.toString();
+ }
+ }
+
+ private static class FeedHealthDataParser implements IMessageAnalyzer {
+
+ private LinkedBlockingQueue<String> inbox = new LinkedBlockingQueue<String>();
+
+ @Override
+ public LinkedBlockingQueue<String> getMessageQueue() {
+ return inbox;
+ }
+
+ }
+
+ private static class FeedJobNotificationHandler implements Runnable, Serializable {
+
+ private static final long serialVersionUID = 1L;
+ private LinkedBlockingQueue<Message> inbox;
+ private Map<JobId, FeedInfo> registeredFeeds = new HashMap<JobId, FeedInfo>();
+ private FeedMessenger feedMessenger;
+ private LinkedBlockingQueue<FeedMessengerMessage> messengerOutbox;
+ private int superFeedManagerPort = 3000;
+
+ public FeedJobNotificationHandler(LinkedBlockingQueue<Message> inbox) {
+ this.inbox = inbox;
+ messengerOutbox = new LinkedBlockingQueue<FeedMessengerMessage>();
+ feedMessenger = new FeedMessenger(messengerOutbox);
+ (new Thread(feedMessenger)).start();
+ }
+
+ public boolean isRegisteredFeed(JobId jobId) {
+ return registeredFeeds.containsKey(jobId);
+ }
+
+ public void registerFeed(FeedConnectionId feedId, JobId jobId, JobSpecification jobSpec,
+ Map<String, String> feedPolicy) {
+ if (registeredFeeds.containsKey(jobId)) {
+ throw new IllegalStateException(" Feed already registered ");
+ }
+ registeredFeeds.put(jobId, new FeedInfo(feedId, jobSpec, feedPolicy, jobId));
+ }
+
+ public void deregisterFeed(JobId jobId) {
+ FeedInfo feedInfo = registeredFeeds.remove(jobId);
+ if (feedInfo != null) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("DeRegistered Feed Info :" + feedInfo);
+ }
+ }
+ }
+
+ public void deregisterFeed(FeedInfo feedInfo) {
+ JobId jobId = feedInfo.jobId;
+ deregisterFeed(jobId);
+ }
+
+ @Override
+ public void run() {
+ Message mesg;
+ while (true) {
+ try {
+ mesg = inbox.take();
+ FeedInfo feedInfo = registeredFeeds.get(mesg.jobId);
+ switch (mesg.messageKind) {
+ case JOB_START:
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Job started for feed id" + feedInfo.feedConnectionId);
+ }
+ handleJobStartMessage(feedInfo, mesg);
+ break;
+ case JOB_FINISH:
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Job finished for feed id" + feedInfo.feedConnectionId);
+ }
+ handleJobFinishMessage(feedInfo, mesg);
+ deregisterFeed(mesg.jobId);
+ break;
+ }
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+
+ }
+ }
+
+ private void handleJobStartMessage(FeedInfo feedInfo, Message message) {
+
+ JobSpecification jobSpec = feedInfo.jobSpec;
+
+ List<OperatorDescriptorId> ingestOperatorIds = new ArrayList<OperatorDescriptorId>();
+ List<OperatorDescriptorId> computeOperatorIds = new ArrayList<OperatorDescriptorId>();
+ List<OperatorDescriptorId> storageOperatorIds = new ArrayList<OperatorDescriptorId>();
+
+ Map<OperatorDescriptorId, IOperatorDescriptor> operators = jobSpec.getOperatorMap();
+ for (Entry<OperatorDescriptorId, IOperatorDescriptor> entry : operators.entrySet()) {
+ IOperatorDescriptor opDesc = entry.getValue();
+ IOperatorDescriptor actualOp = null;
+ if (opDesc instanceof FeedMetaOperatorDescriptor) {
+ actualOp = ((FeedMetaOperatorDescriptor) opDesc).getCoreOperator();
+ } else {
+ actualOp = opDesc;
+ }
+
+ if (actualOp instanceof AlgebricksMetaOperatorDescriptor) {
+ AlgebricksMetaOperatorDescriptor op = ((AlgebricksMetaOperatorDescriptor) actualOp);
+ IPushRuntimeFactory[] runtimeFactories = op.getPipeline().getRuntimeFactories();
+ for (IPushRuntimeFactory rf : runtimeFactories) {
+ if (rf instanceof AssignRuntimeFactory) {
+ computeOperatorIds.add(entry.getKey());
+ }
+ }
+ } else if (actualOp instanceof LSMTreeIndexInsertUpdateDeleteOperatorDescriptor) {
+ storageOperatorIds.add(entry.getKey());
+ } else if (actualOp instanceof FeedIntakeOperatorDescriptor) {
+ ingestOperatorIds.add(entry.getKey());
+ }
+ }
+
+ try {
+ IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
+ JobInfo info = hcc.getJobInfo(message.jobId);
+ feedInfo.jobInfo = info;
+ Map<String, String> feedActivityDetails = new HashMap<String, String>();
+ StringBuilder ingestLocs = new StringBuilder();
+ for (OperatorDescriptorId ingestOpId : ingestOperatorIds) {
+ feedInfo.ingestLocations.addAll(info.getOperatorLocations().get(ingestOpId));
+ }
+ StringBuilder computeLocs = new StringBuilder();
+ for (OperatorDescriptorId computeOpId : computeOperatorIds) {
+ List<String> locations = info.getOperatorLocations().get(computeOpId);
+ if (locations != null) {
+ feedInfo.computeLocations.addAll(locations);
+ } else {
+ feedInfo.computeLocations.addAll(feedInfo.ingestLocations);
+ }
+ }
+ StringBuilder storageLocs = new StringBuilder();
+ for (OperatorDescriptorId storageOpId : storageOperatorIds) {
+ feedInfo.storageLocations.addAll(info.getOperatorLocations().get(storageOpId));
+ }
+
+ for (String ingestLoc : feedInfo.ingestLocations) {
+ ingestLocs.append(ingestLoc);
+ ingestLocs.append(",");
+ }
+ if (ingestLocs.length() > 1) {
+ ingestLocs.deleteCharAt(ingestLocs.length() - 1);
+ }
+ for (String computeLoc : feedInfo.computeLocations) {
+ computeLocs.append(computeLoc);
+ computeLocs.append(",");
+ }
+ if (computeLocs.length() > 1) {
+ computeLocs.deleteCharAt(computeLocs.length() - 1);
+ }
+ for (String storageLoc : feedInfo.storageLocations) {
+ storageLocs.append(storageLoc);
+ storageLocs.append(",");
+ }
+ if (storageLocs.length() > 1) {
+ storageLocs.deleteCharAt(storageLocs.length() - 1);
+ }
+
+ feedActivityDetails.put(FeedActivity.FeedActivityDetails.INGEST_LOCATIONS, ingestLocs.toString());
+ feedActivityDetails.put(FeedActivity.FeedActivityDetails.COMPUTE_LOCATIONS, computeLocs.toString());
+ feedActivityDetails.put(FeedActivity.FeedActivityDetails.STORAGE_LOCATIONS, storageLocs.toString());
+ feedActivityDetails.put(FeedActivity.FeedActivityDetails.FEED_POLICY_NAME,
+ feedInfo.feedPolicy.get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY));
+
+ int superFeedManagerIndex = new Random().nextInt(feedInfo.ingestLocations.size());
+ String superFeedManagerHost = feedInfo.ingestLocations.get(superFeedManagerIndex);
+
+ Cluster cluster = AsterixClusterProperties.INSTANCE.getCluster();
+ String instanceName = cluster.getInstanceName();
+ String node = superFeedManagerHost.substring(instanceName.length() + 1);
+ String hostIp = null;
+ for (Node n : cluster.getNode()) {
+ if (n.getId().equals(node)) {
+ hostIp = n.getClusterIp();
+ break;
+ }
+ }
+ if (hostIp == null) {
+ throw new IllegalStateException("Unknown node " + superFeedManagerHost);
+ }
+
+ feedActivityDetails.put(FeedActivity.FeedActivityDetails.SUPER_FEED_MANAGER_HOST, hostIp);
+ feedActivityDetails.put(FeedActivity.FeedActivityDetails.SUPER_FEED_MANAGER_PORT, ""
+ + superFeedManagerPort);
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Super Feed Manager for " + feedInfo.feedConnectionId + " is " + hostIp + " node "
+ + superFeedManagerHost);
+ }
+
+ FeedManagerElectMessage feedMessage = new FeedManagerElectMessage(hostIp, superFeedManagerHost,
+ superFeedManagerPort, feedInfo.feedConnectionId);
+ superFeedManagerPort += SuperFeedManager.PORT_RANGE_ASSIGNED;
+ messengerOutbox.add(new FeedMessengerMessage(feedMessage, feedInfo));
+ MetadataManager.INSTANCE.acquireWriteLatch();
+ MetadataTransactionContext mdTxnCtx = null;
+ try {
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ FeedActivity fa = MetadataManager.INSTANCE.getRecentActivityOnFeedConnection(mdTxnCtx,
+ feedInfo.feedConnectionId, null);
+ FeedActivityType nextState = FeedActivityType.FEED_BEGIN;
+ FeedActivity feedActivity = new FeedActivity(feedInfo.feedConnectionId.getDataverse(),
+ feedInfo.feedConnectionId.getFeedName(), feedInfo.feedConnectionId.getDatasetName(),
+ nextState, feedActivityDetails);
+ MetadataManager.INSTANCE.registerFeedActivity(mdTxnCtx, feedInfo.feedConnectionId, feedActivity);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (Exception e) {
+ MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+ } finally {
+ MetadataManager.INSTANCE.releaseWriteLatch();
+ }
+ } catch (Exception e) {
+ // TODO Add Exception handling here
+ }
+
+ }
+
+ private void handleJobFinishMessage(FeedInfo feedInfo, Message message) {
+ MetadataManager.INSTANCE.acquireWriteLatch();
+ MetadataTransactionContext mdTxnCtx = null;
+ boolean feedFailedDueToPostSubmissionNodeLoss = verfyReasonForFailure(feedInfo);
+ if (!feedFailedDueToPostSubmissionNodeLoss) {
+ try {
+ IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
+ JobInfo info = hcc.getJobInfo(message.jobId);
+ JobStatus status = info.getPendingStatus();
+ List<Exception> exceptions;
+ boolean failure = status != null && status.equals(JobStatus.FAILURE);
+ FeedActivityType activityType = FeedActivityType.FEED_END;
+ Map<String, String> details = new HashMap<String, String>();
+ if (failure) {
+ exceptions = info.getPendingExceptions();
+ activityType = FeedActivityType.FEED_FAILURE;
+ details.put(FeedActivity.FeedActivityDetails.EXCEPTION_MESSAGE, exceptions.get(0).getMessage());
+ }
+ mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ FeedActivity feedActivity = new FeedActivity(feedInfo.feedConnectionId.getDataverse(),
+ feedInfo.feedConnectionId.getFeedName(), feedInfo.feedConnectionId.getDatasetName(),
+ activityType, details);
+ MetadataManager.INSTANCE.registerFeedActivity(mdTxnCtx, new FeedConnectionId(
+ feedInfo.feedConnectionId.getDataverse(), feedInfo.feedConnectionId.getFeedName(),
+ feedInfo.feedConnectionId.getDatasetName()), feedActivity);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (RemoteException | ACIDException | MetadataException e) {
+ try {
+ MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+ } catch (RemoteException | ACIDException ae) {
+ throw new IllegalStateException(" Unable to abort ");
+ }
+ } catch (Exception e) {
+ // add exception handling here
+ } finally {
+ MetadataManager.INSTANCE.releaseWriteLatch();
+ }
+ } else {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Attempt to revive feed");
+ }
+ FeedsActivator activator = new FeedsActivator();
+ String dataverse = feedInfo.feedConnectionId.getDataverse();
+ String datasetName = feedInfo.feedConnectionId.getDatasetName();
+ String feedName = feedInfo.feedConnectionId.getFeedName();
+ String feedPolicy = feedInfo.feedPolicy.get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY);
+ activator.reviveFeed(dataverse, feedName, datasetName, feedPolicy);
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Revived Feed");
+ }
+
+ }
+ }
+
+ private boolean verfyReasonForFailure(FeedInfo feedInfo) {
+ JobSpecification spec = feedInfo.jobSpec;
+ Set<Constraint> userConstraints = spec.getUserConstraints();
+ List<String> locations = new ArrayList<String>();
+ for (Constraint constraint : userConstraints) {
+ LValueConstraintExpression lexpr = constraint.getLValue();
+ ConstraintExpression cexpr = constraint.getRValue();
+ switch (lexpr.getTag()) {
+ case PARTITION_LOCATION:
+ String location = (String) ((ConstantExpression) cexpr).getValue();
+ locations.add(location);
+ break;
+ }
+ }
+ Set<String> participantNodes = AsterixClusterProperties.INSTANCE.getParticipantNodes();
+ List<String> nodesFailedPostSubmission = new ArrayList<String>();
+ for (String location : locations) {
+ if (!participantNodes.contains(location)) {
+ nodesFailedPostSubmission.add(location);
+ }
+ }
+
+ if (nodesFailedPostSubmission.size() > 0) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Feed failed as nodes failed post submission");
+ }
+ return true;
+ } else {
+ return false;
+ }
+
+ }
+
+ public static class FeedMessengerMessage {
+ private final IFeedMessage message;
+ private final FeedInfo feedInfo;
+
+ public FeedMessengerMessage(IFeedMessage message, FeedInfo feedInfo) {
+ this.message = message;
+ this.feedInfo = feedInfo;
+ }
+
+ public IFeedMessage getMessage() {
+ return message;
+ }
+
+ public FeedInfo getFeedInfo() {
+ return feedInfo;
+ }
+ }
+
+ private static class FeedMessenger implements Runnable {
+
+ private final LinkedBlockingQueue<FeedMessengerMessage> inbox;
+
+ public FeedMessenger(LinkedBlockingQueue<FeedMessengerMessage> inbox) {
+ this.inbox = inbox;
+ }
+
+ public void run() {
+ while (true) {
+ FeedMessengerMessage message = null;
+ try {
+ message = inbox.take();
+ FeedInfo feedInfo = message.getFeedInfo();
+ switch (message.getMessage().getMessageType()) {
+ case SUPER_FEED_MANAGER_ELECT:
+ Thread.sleep(2000);
+ sendSuperFeedManangerElectMessage(feedInfo,
+ (FeedManagerElectMessage) message.getMessage());
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Sent super feed manager election message" + message.getMessage());
+ }
+ }
+ } catch (InterruptedException ie) {
+ break;
+ }
+ }
+ }
+
+ }
+ }
+
+ public static class FeedInfo {
+ public FeedConnectionId feedConnectionId;
+ public JobSpecification jobSpec;
+ public List<String> ingestLocations = new ArrayList<String>();
+ public List<String> computeLocations = new ArrayList<String>();
+ public List<String> storageLocations = new ArrayList<String>();
+ public JobInfo jobInfo;
+ public Map<String, String> feedPolicy;
+ public JobId jobId;
+
+ public FeedInfo(FeedConnectionId feedId, JobSpecification jobSpec, Map<String, String> feedPolicy, JobId jobId) {
+ this.feedConnectionId = feedId;
+ this.jobSpec = jobSpec;
+ this.feedPolicy = feedPolicy;
+ this.jobId = jobId;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (!(o instanceof FeedInfo)) {
+ return false;
+ }
+ return ((FeedInfo) o).feedConnectionId.equals(feedConnectionId);
+ }
+
+ @Override
+ public int hashCode() {
+ return feedConnectionId.hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return feedConnectionId + " job id " + jobId;
+ }
+
+ }
+
+ @Override
+ public Set<IClusterManagementWork> notifyNodeFailure(Set<String> deadNodeIds) {
+ Collection<FeedInfo> feedInfos = feedJobNotificationHandler.registeredFeeds.values();
+ FeedFailureReport failureReport = new FeedFailureReport();
+ for (FeedInfo feedInfo : feedInfos) {
+ for (String deadNodeId : deadNodeIds) {
+ if (feedInfo.ingestLocations.contains(deadNodeId)) {
+ List<FeedFailure> failures = failureReport.failures.get(feedInfo);
+ if (failures == null) {
+ failures = new ArrayList<FeedFailure>();
+ failureReport.failures.put(feedInfo, failures);
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Inestion Node Failure! " + deadNodeId);
+ }
+ failures.add(new FeedFailure(FeedFailure.FailureType.INGESTION_NODE, deadNodeId));
+ }
+ if (feedInfo.computeLocations.contains(deadNodeId)) {
+ List<FeedFailure> failures = failureReport.failures.get(feedInfo);
+ if (failures == null) {
+ failures = new ArrayList<FeedFailure>();
+ failureReport.failures.put(feedInfo, failures);
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Compute Node Failure! " + deadNodeId);
+ }
+ failures.add(new FeedFailure(FeedFailure.FailureType.COMPUTE_NODE, deadNodeId));
+ }
+ if (feedInfo.storageLocations.contains(deadNodeId)) {
+ List<FeedFailure> failures = failureReport.failures.get(feedInfo);
+ if (failures == null) {
+ failures = new ArrayList<FeedFailure>();
+ failureReport.failures.put(feedInfo, failures);
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Storage Node Failure! " + deadNodeId);
+ }
+ failures.add(new FeedFailure(FeedFailure.FailureType.STORAGE_NODE, deadNodeId));
+ }
+ }
+ }
+ if (failureReport.failures.isEmpty()) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ StringBuilder builder = new StringBuilder();
+ builder.append("No feed is affected by the failure of node(s): ");
+ for (String deadNodeId : deadNodeIds) {
+ builder.append(deadNodeId + " ");
+ }
+ LOGGER.info(builder.toString());
+ }
+ return new HashSet<IClusterManagementWork>();
+ } else {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ StringBuilder builder = new StringBuilder();
+ builder.append("Feed affected by the failure of node(s): ");
+ for (String deadNodeId : deadNodeIds) {
+ builder.append(deadNodeId + " ");
+ }
+ builder.append("\n");
+ for (FeedInfo fInfo : failureReport.failures.keySet()) {
+ builder.append(fInfo.feedConnectionId);
+ feedJobNotificationHandler.deregisterFeed(fInfo);
+ }
+ LOGGER.warning(builder.toString());
+ }
+ return handleFailure(failureReport);
+ }
+ }
+
+ private Set<IClusterManagementWork> handleFailure(FeedFailureReport failureReport) {
+ reportFeedFailure(failureReport);
+ Set<IClusterManagementWork> work = new HashSet<IClusterManagementWork>();
+ Map<String, Map<FeedInfo, List<FailureType>>> failureMap = new HashMap<String, Map<FeedInfo, List<FailureType>>>();
+ FeedPolicyAccessor fpa = null;
+ List<FeedInfo> feedsToTerminate = new ArrayList<FeedInfo>();
+ for (Map.Entry<FeedInfo, List<FeedFailure>> entry : failureReport.failures.entrySet()) {
+ FeedInfo feedInfo = entry.getKey();
+ fpa = new FeedPolicyAccessor(feedInfo.feedPolicy);
+ if (!fpa.continueOnHardwareFailure()) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Feed " + feedInfo.feedConnectionId + " is governed by policy "
+ + feedInfo.feedPolicy.get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY));
+ LOGGER.warning("Feed policy does not require feed to recover from hardware failure. Feed will terminate");
+ }
+ continue;
+ } else {
+ // insert feed recovery mode
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Feed " + feedInfo.feedConnectionId + " is governed by policy "
+ + feedInfo.feedPolicy.get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY));
+ LOGGER.info("Feed policy requires feed to recover from hardware failure. Attempting to recover feed");
+ }
+ }
+
+ List<FeedFailure> feedFailures = entry.getValue();
+ boolean recoveryPossible = true;
+ for (FeedFailure feedFailure : feedFailures) {
+ switch (feedFailure.failureType) {
+ case COMPUTE_NODE:
+ case INGESTION_NODE:
+ Map<FeedInfo, List<FailureType>> failuresBecauseOfThisNode = failureMap.get(feedFailure.nodeId);
+ if (failuresBecauseOfThisNode == null) {
+ failuresBecauseOfThisNode = new HashMap<FeedInfo, List<FailureType>>();
+ failuresBecauseOfThisNode.put(feedInfo, new ArrayList<FailureType>());
+ failureMap.put(feedFailure.nodeId, failuresBecauseOfThisNode);
+ }
+ List<FailureType> feedF = failuresBecauseOfThisNode.get(feedInfo);
+ if (feedF == null) {
+ feedF = new ArrayList<FailureType>();
+ failuresBecauseOfThisNode.put(feedInfo, feedF);
+ }
+ feedF.add(feedFailure.failureType);
+ break;
+ case STORAGE_NODE:
+ recoveryPossible = false;
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("Unrecoverable situation! lost storage node for the feed "
+ + feedInfo.feedConnectionId);
+ }
+ List<String> requiredNodeIds = dependentFeeds.get(feedInfo);
+ if (requiredNodeIds == null) {
+ requiredNodeIds = new ArrayList<String>();
+ dependentFeeds.put(feedInfo, requiredNodeIds);
+ }
+ requiredNodeIds.add(feedFailure.nodeId);
+ failuresBecauseOfThisNode = failureMap.get(feedFailure.nodeId);
+ if (failuresBecauseOfThisNode != null) {
+ failuresBecauseOfThisNode.remove(feedInfo);
+ if (failuresBecauseOfThisNode.isEmpty()) {
+ failureMap.remove(feedFailure.nodeId);
+ }
+ }
+ feedsToTerminate.add(feedInfo);
+ break;
+ }
+ }
+ if (!recoveryPossible) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Terminating irrecoverable feed (loss of storage node) ");
+ }
+ }
+ }
+
+ if (!feedsToTerminate.isEmpty()) {
+ Thread t = new Thread(new FeedsDeActivator(feedsToTerminate));
+ t.start();
+ }
+
+ int numRequiredNodes = 0;
+ for (Entry<String, Map<FeedInfo, List<FeedFailure.FailureType>>> entry : failureMap.entrySet()) {
+ Map<FeedInfo, List<FeedFailure.FailureType>> v = entry.getValue();
+ for (FeedInfo finfo : feedsToTerminate) {
+ v.remove(finfo);
+ }
+ if (v.size() > 0) {
+ numRequiredNodes++;
+ }
+ }
+
+ if (numRequiredNodes > 0) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Number of additional nodes requested " + numRequiredNodes);
+ }
+ AddNodeWork addNodesWork = new AddNodeWork(failureMap.keySet().size(), this);
+ work.add(addNodesWork);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ Map<FeedInfo, List<FeedFailure>> feedFailures = failureReport.failures;
+ for (Entry<FeedInfo, List<FeedFailure>> entry : feedFailures.entrySet()) {
+ for (FeedFailure f : entry.getValue()) {
+ LOGGER.info("Feed Failure! " + f.failureType + " " + f.nodeId);
+ }
+ }
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Registered work id: " + addNodesWork.getWorkId());
+ }
+ feedWorkRequestResponseHandler.registerFeedWork(addNodesWork.getWorkId(), failureReport);
+ } else {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Not requesting any new node. Feeds unrecoverable until the lost node(s) rejoin");
+ }
+ }
+ return work;
+ }
+
+ private void reportFeedFailure(FeedFailureReport failureReport) {
+ MetadataTransactionContext ctx = null;
+ FeedActivity fa = null;
+ Map<String, String> feedActivityDetails = new HashMap<String, String>();
+ StringBuilder builder = new StringBuilder();
+ MetadataManager.INSTANCE.acquireWriteLatch();
+ try {
+ ctx = MetadataManager.INSTANCE.beginTransaction();
+ for (Entry<FeedInfo, List<FeedFailure>> entry : failureReport.failures.entrySet()) {
+ FeedInfo feedInfo = entry.getKey();
+ List<FeedFailure> feedFailures = entry.getValue();
+ for (FeedFailure failure : feedFailures) {
+ builder.append(failure + ",");
+ }
+ builder.deleteCharAt(builder.length() - 1);
+ feedActivityDetails.put(FeedActivityDetails.FEED_NODE_FAILURE, builder.toString());
+ fa = new FeedActivity(feedInfo.feedConnectionId.getDataverse(),
+ feedInfo.feedConnectionId.getFeedName(), feedInfo.feedConnectionId.getDatasetName(),
+ FeedActivityType.FEED_FAILURE, feedActivityDetails);
+ MetadataManager.INSTANCE.registerFeedActivity(ctx, feedInfo.feedConnectionId, fa);
+ }
+ MetadataManager.INSTANCE.commitTransaction(ctx);
+ } catch (Exception e) {
+ if (ctx != null) {
+ try {
+ MetadataManager.INSTANCE.abortTransaction(ctx);
+ } catch (Exception e2) {
+ e2.addSuppressed(e);
+ throw new IllegalStateException("Unable to abort transaction " + e2);
+ }
+ }
+ } finally {
+ MetadataManager.INSTANCE.releaseWriteLatch();
+ }
+ }
+
+ private static void sendSuperFeedManangerElectMessage(FeedInfo feedInfo, FeedManagerElectMessage electMessage) {
+ try {
+ Dataverse dataverse = new Dataverse(feedInfo.feedConnectionId.getDataverse(),
+ NonTaggedDataFormat.NON_TAGGED_DATA_FORMAT, 0);
+ AqlMetadataProvider metadataProvider = new AqlMetadataProvider(dataverse);
+ JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+
+ IOperatorDescriptor feedMessenger;
+ AlgebricksPartitionConstraint messengerPc;
+ Set<String> locations = new HashSet<String>();
+ locations.addAll(feedInfo.computeLocations);
+ locations.addAll(feedInfo.ingestLocations);
+ locations.addAll(feedInfo.storageLocations);
+
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = metadataProvider.buildSendFeedMessageRuntime(
+ spec, dataverse.getDataverseName(), feedInfo.feedConnectionId.getFeedName(),
+ feedInfo.feedConnectionId.getDatasetName(), electMessage, locations.toArray(new String[] {}));
+ feedMessenger = p.first;
+ messengerPc = p.second;
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, feedMessenger, messengerPc);
+
+ NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(spec);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, nullSink, messengerPc);
+ spec.connect(new OneToOneConnectorDescriptor(spec), feedMessenger, 0, nullSink, 0);
+ spec.addRoot(nullSink);
+
+ JobId jobId = AsterixAppContextInfo.getInstance().getHcc().startJob(spec);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(" Super Feed Manager Message: " + electMessage + " Job Id " + jobId);
+ }
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Exception in sending super feed manager elect message: " + feedInfo.feedConnectionId + " "
+ + e.getMessage());
+ }
+ }
+ }
+
+ public static class FeedFailure {
+
+ public enum FailureType {
+ INGESTION_NODE,
+ COMPUTE_NODE,
+ STORAGE_NODE
+ }
+
+ public FailureType failureType;
+ public String nodeId;
+
+ public FeedFailure(FailureType failureType, String nodeId) {
+ this.failureType = failureType;
+ this.nodeId = nodeId;
+ }
+
+ @Override
+ public String toString() {
+ return failureType + " (" + nodeId + ") ";
+ }
+ }
+
+ @Override
+ public Set<IClusterManagementWork> notifyNodeJoin(String joinedNodeId) {
+ State newState = AsterixClusterProperties.INSTANCE.getState();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(joinedNodeId + " joined the cluster. " + "Asterix state: " + newState);
+ }
+
+ boolean needToReActivateFeeds = !newState.equals(state) && (newState == State.ACTIVE);
+ if (needToReActivateFeeds) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(joinedNodeId + " Resuming loser feeds (if any)");
+ }
+ try {
+ FeedsActivator activator = new FeedsActivator();
+ (new Thread(activator)).start();
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Exception in resuming feeds" + e.getMessage());
+ }
+ }
+ state = newState;
+ } else {
+ List<FeedInfo> feedsThatCanBeRevived = new ArrayList<FeedInfo>();
+ for (Entry<FeedInfo, List<String>> entry : dependentFeeds.entrySet()) {
+ List<String> requiredNodeIds = entry.getValue();
+ if (requiredNodeIds.contains(joinedNodeId)) {
+ requiredNodeIds.remove(joinedNodeId);
+ if (requiredNodeIds.isEmpty()) {
+ feedsThatCanBeRevived.add(entry.getKey());
+ }
+ }
+ }
+ if (!feedsThatCanBeRevived.isEmpty()) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(joinedNodeId + " Resuming feeds after rejoining of node " + joinedNodeId);
+ }
+ FeedsActivator activator = new FeedsActivator(feedsThatCanBeRevived);
+ (new Thread(activator)).start();
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public void notifyRequestCompletion(IClusterManagementWorkResponse response) {
+ try {
+ responseInbox.put(response);
+ } catch (InterruptedException e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Interrupted exception");
+ }
+ }
+ }
+
+ @Override
+ public void notifyStateChange(State previousState, State newState) {
+ switch (newState) {
+ case ACTIVE:
+ if (previousState.equals(State.UNUSABLE)) {
+ try {
+ FeedsActivator activator = new FeedsActivator();
+ (new Thread(activator)).start();
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Exception in resuming feeds" + e.getMessage());
+ }
+ }
+ }
+ break;
+ }
+
+ }
+
+ private static class FeedsActivator implements Runnable {
+
+ private List<FeedInfo> feedsToRevive;
+ private Mode mode;
+
+ public enum Mode {
+ REVIVAL_POST_CLUSTER_REBOOT,
+ REVIVAL_POST_NODE_REJOIN
+ }
+
+ public FeedsActivator() {
+ this.mode = Mode.REVIVAL_POST_CLUSTER_REBOOT;
+ }
+
+ public FeedsActivator(List<FeedInfo> feedsToRevive) {
+ this.feedsToRevive = feedsToRevive;
+ this.mode = Mode.REVIVAL_POST_NODE_REJOIN;
+ }
+
+ @Override
+ public void run() {
+ switch (mode) {
+ case REVIVAL_POST_CLUSTER_REBOOT:
+ revivePostClusterReboot();
+ break;
+ case REVIVAL_POST_NODE_REJOIN:
+ try {
+ Thread.sleep(10000);
+ } catch (InterruptedException e1) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Attempt to resume feed interrupted");
+ }
+ throw new IllegalStateException(e1.getMessage());
+ }
+ for (FeedInfo finfo : feedsToRevive) {
+ try {
+ JobId jobId = AsterixAppContextInfo.getInstance().getHcc().startJob(finfo.jobSpec);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Resumed feed :" + finfo.feedConnectionId + " job id " + jobId);
+ LOGGER.info("Job:" + finfo.jobSpec);
+ }
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to resume feed " + finfo.feedConnectionId + " " + e.getMessage());
+ }
+ }
+ }
+ }
+ }
+
+ private void revivePostClusterReboot() {
+ MetadataTransactionContext ctx = null;
+
+ try {
+
+ Thread.sleep(4000);
+ MetadataManager.INSTANCE.init();
+ ctx = MetadataManager.INSTANCE.beginTransaction();
+ List<FeedActivity> activeFeeds = MetadataManager.INSTANCE.getActiveFeeds(ctx, null, null);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Attempt to resume feeds that were active prior to instance shutdown!");
+ LOGGER.info("Number of feeds affected:" + activeFeeds.size());
+ for (FeedActivity fa : activeFeeds) {
+ LOGGER.info("Active feed " + fa.getDataverseName() + ":" + fa.getDatasetName());
+ }
+ }
+ for (FeedActivity fa : activeFeeds) {
+ String feedPolicy = fa.getFeedActivityDetails().get(FeedActivityDetails.FEED_POLICY_NAME);
+ FeedPolicy policy = MetadataManager.INSTANCE.getFeedPolicy(ctx, fa.getDataverseName(), feedPolicy);
+ if (policy == null) {
+ policy = MetadataManager.INSTANCE.getFeedPolicy(ctx, MetadataConstants.METADATA_DATAVERSE_NAME,
+ feedPolicy);
+ if (policy == null) {
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("Unable to resume feed: " + fa.getDataverseName() + ":"
+ + fa.getDatasetName() + "." + " Unknown policy :" + feedPolicy);
+ }
+ continue;
+ }
+ }
+
+ FeedPolicyAccessor fpa = new FeedPolicyAccessor(policy.getProperties());
+ if (fpa.autoRestartOnClusterReboot()) {
+ String dataverse = fa.getDataverseName();
+ String datasetName = fa.getDatasetName();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Resuming feed after cluster revival: " + dataverse + ":" + datasetName
+ + " using policy " + feedPolicy);
+ }
+ reviveFeed(dataverse, fa.getFeedName(), datasetName, feedPolicy);
+ } else {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Feed " + fa.getDataverseName() + ":" + fa.getDatasetName()
+ + " governed by policy" + feedPolicy
+ + " does not state auto restart after cluster revival");
+ }
+ }
+ }
+ MetadataManager.INSTANCE.commitTransaction(ctx);
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ try {
+ MetadataManager.INSTANCE.abortTransaction(ctx);
+ } catch (Exception e1) {
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("Exception in aborting" + e.getMessage());
+ }
+ throw new IllegalStateException(e1);
+ }
+ }
+ }
+
+ public void reviveFeed(String dataverse, String feedName, String dataset, String feedPolicy) {
+ PrintWriter writer = new PrintWriter(System.out, true);
+ SessionConfig pc = new SessionConfig(true, false, false, false, false, false, true, true, false);
+ try {
+ DataverseDecl dataverseDecl = new DataverseDecl(new Identifier(dataverse));
+ ConnectFeedStatement stmt = new ConnectFeedStatement(new Identifier(dataverse),
+ new Identifier(feedName), new Identifier(dataset), feedPolicy, 0);
+ stmt.setForceConnect(true);
+ List<Statement> statements = new ArrayList<Statement>();
+ statements.add(dataverseDecl);
+ statements.add(stmt);
+ AqlTranslator translator = new AqlTranslator(statements, writer, pc, DisplayFormat.TEXT);
+ translator.compileAndExecute(AsterixAppContextInfo.getInstance().getHcc(), null, false);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Resumed feed: " + dataverse + ":" + dataset + " using policy " + feedPolicy);
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Exception in resuming loser feed: " + dataverse + ":" + dataset + " using policy "
+ + feedPolicy + " Exception " + e.getMessage());
+ }
+ }
+ }
+ }
+
+ public static class FeedsDeActivator implements Runnable {
+
+ private List<FeedInfo> feedsToTerminate;
+
+ public FeedsDeActivator(List<FeedInfo> feedsToTerminate) {
+ this.feedsToTerminate = feedsToTerminate;
+ }
+
+ @Override
+ public void run() {
+ for (FeedInfo feedInfo : feedsToTerminate) {
+ endFeed(feedInfo);
+ }
+ }
+
+ private void endFeed(FeedInfo feedInfo) {
+ MetadataTransactionContext ctx = null;
+ PrintWriter writer = new PrintWriter(System.out, true);
+ SessionConfig pc = new SessionConfig(true, false, false, false, false, false, true, true, false);
+ try {
+ ctx = MetadataManager.INSTANCE.beginTransaction();
+ DisconnectFeedStatement stmt = new DisconnectFeedStatement(new Identifier(
+ feedInfo.feedConnectionId.getDataverse()), new Identifier(
+ feedInfo.feedConnectionId.getFeedName()), new Identifier(
+ feedInfo.feedConnectionId.getDatasetName()));
+ List<Statement> statements = new ArrayList<Statement>();
+ DataverseDecl dataverseDecl = new DataverseDecl(
+ new Identifier(feedInfo.feedConnectionId.getDataverse()));
+ statements.add(dataverseDecl);
+ statements.add(stmt);
+ AqlTranslator translator = new AqlTranslator(statements, writer, pc, DisplayFormat.TEXT);
+ translator.compileAndExecute(AsterixAppContextInfo.getInstance().getHcc(), null, false);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("End urecoverable feed: " + feedInfo.feedConnectionId);
+ }
+ MetadataManager.INSTANCE.commitTransaction(ctx);
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Exception in ending loser feed: " + feedInfo.feedConnectionId + " Exception "
+ + e.getMessage());
+ }
+ e.printStackTrace();
+ try {
+ MetadataManager.INSTANCE.abortTransaction(ctx);
+ } catch (Exception e2) {
+ e2.addSuppressed(e);
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("Exception in aborting transaction! System is in inconsistent state");
+ }
+ }
+
+ }
+
+ }
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedWorkRequestResponseHandler.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedWorkRequestResponseHandler.java
new file mode 100644
index 0000000..3197bbb
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedWorkRequestResponseHandler.java
@@ -0,0 +1,340 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.hyracks.bootstrap;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.hyracks.bootstrap.FeedLifecycleListener.FeedFailure;
+import edu.uci.ics.asterix.hyracks.bootstrap.FeedLifecycleListener.FeedFailureReport;
+import edu.uci.ics.asterix.hyracks.bootstrap.FeedLifecycleListener.FeedInfo;
+import edu.uci.ics.asterix.hyracks.bootstrap.FeedLifecycleListener.FeedsDeActivator;
+import edu.uci.ics.asterix.metadata.api.IClusterManagementWork;
+import edu.uci.ics.asterix.metadata.cluster.AddNodeWork;
+import edu.uci.ics.asterix.metadata.cluster.AddNodeWorkResponse;
+import edu.uci.ics.asterix.metadata.cluster.IClusterManagementWorkResponse;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
+import edu.uci.ics.hyracks.api.constraints.Constraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression.ExpressionTag;
+import edu.uci.ics.hyracks.api.constraints.expressions.LValueConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.PartitionCountExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.PartitionLocationExpression;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class FeedWorkRequestResponseHandler implements Runnable {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedWorkRequestResponseHandler.class.getName());
+
+ private final LinkedBlockingQueue<IClusterManagementWorkResponse> inbox;
+
+ private Map<Integer, FeedFailureReport> feedsWaitingForResponse = new HashMap<Integer, FeedFailureReport>();
+
+ public FeedWorkRequestResponseHandler(LinkedBlockingQueue<IClusterManagementWorkResponse> inbox) {
+ this.inbox = inbox;
+ }
+
+ @Override
+ public void run() {
+ while (true) {
+ IClusterManagementWorkResponse response = null;
+ try {
+ response = inbox.take();
+ } catch (InterruptedException e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Interrupted exception " + e.getMessage());
+ }
+ }
+ IClusterManagementWork submittedWork = response.getWork();
+ switch (submittedWork.getClusterManagementWorkType()) {
+ case ADD_NODE:
+ AddNodeWorkResponse resp = (AddNodeWorkResponse) response;
+ switch (resp.getStatus()) {
+ case FAILURE:
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Request " + resp.getWork() + " not completed");
+ }
+ break;
+ case SUCCESS:
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Request " + resp.getWork() + " completed");
+ }
+ break;
+ }
+
+ AddNodeWork work = (AddNodeWork) submittedWork;
+ FeedFailureReport failureReport = feedsWaitingForResponse.remove(work.getWorkId());
+ Set<FeedInfo> affectedFeeds = failureReport.failures.keySet();
+ for (FeedInfo feedInfo : affectedFeeds) {
+ try {
+ recoverFeed(feedInfo, work, resp, failureReport.failures.get(feedInfo));
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Recovered feed:" + feedInfo);
+ }
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("Unable to recover feed:" + feedInfo);
+ }
+ }
+ }
+ break;
+ case REMOVE_NODE:
+ break;
+ }
+ }
+ }
+
+ private void recoverFeed(FeedInfo feedInfo, AddNodeWork work, AddNodeWorkResponse resp,
+ List<FeedFailure> feedFailures) throws Exception {
+ List<String> failedNodeIds = new ArrayList<String>();
+ for (FeedFailure feedFailure : feedFailures) {
+ failedNodeIds.add(feedFailure.nodeId);
+ }
+ List<String> chosenReplacements = new ArrayList<String>();
+ String metadataNodeName = AsterixAppContextInfo.getInstance().getMetadataProperties().getMetadataNodeName();
+ chosenReplacements.add(metadataNodeName);
+ switch (resp.getStatus()) {
+ case FAILURE:
+ for (FeedFailure feedFailure : feedFailures) {
+ switch (feedFailure.failureType) {
+ case INGESTION_NODE:
+ String replacement = getInternalReplacement(feedInfo, feedFailure, failedNodeIds,
+ chosenReplacements);
+ chosenReplacements.add(replacement);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Existing node:" + replacement + " chosen to replace "
+ + feedFailure.nodeId);
+ }
+ alterFeedJobSpec(feedInfo, resp, feedFailure.nodeId, replacement);
+ break;
+ }
+ }
+ break;
+ case SUCCESS:
+ List<String> nodesAdded = resp.getNodesAdded();
+ int numNodesAdded = nodesAdded.size();
+ int nodeIndex = 0;
+ for (FeedFailure feedFailure : feedFailures) {
+ switch (feedFailure.failureType) {
+ case INGESTION_NODE:
+ String replacement = null;
+ if (nodeIndex <= numNodesAdded - 1) {
+ replacement = nodesAdded.get(nodeIndex);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Newly added node:" + replacement + " chosen to replace "
+ + feedFailure.nodeId);
+ }
+ } else {
+ replacement = getInternalReplacement(feedInfo, feedFailure, failedNodeIds,
+ chosenReplacements);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Existing node:" + replacement + " chosen to replace "
+ + feedFailure.nodeId);
+ }
+ chosenReplacements.add(replacement);
+ }
+ alterFeedJobSpec(feedInfo, resp, feedFailure.nodeId, replacement);
+ nodeIndex++;
+ break;
+ default: // ingestion nodes and compute nodes (in currrent implementation) coincide.
+ // so correcting ingestion node failure also takes care of compute nodes failure.
+ // Storage node failures cannot be recovered from as in current implementation, we
+ // do not have data replication.
+ }
+ }
+ break;
+ }
+
+ JobSpecification spec = feedInfo.jobSpec;
+ System.out.println("Final recovery Job Spec \n" + spec);
+ Thread.sleep(5000);
+ AsterixAppContextInfo.getInstance().getHcc().startJob(feedInfo.jobSpec);
+ }
+
+ private String getInternalReplacement(FeedInfo feedInfo, FeedFailure feedFailure, List<String> failedNodeIds,
+ List<String> chosenReplacements) {
+ String failedNodeId = feedFailure.nodeId;
+ String replacement = null;;
+ // TODO 1st preference is given to any other participant node that is not involved in the feed.
+ // 2nd preference is given to a compute node.
+ // 3rd preference is given to a storage node
+ Set<String> participantNodes = AsterixClusterProperties.INSTANCE.getParticipantNodes();
+ if (participantNodes != null && !participantNodes.isEmpty()) {
+ List<String> pNodesClone = new ArrayList<String>();
+ pNodesClone.addAll(participantNodes);
+ pNodesClone.removeAll(feedInfo.storageLocations);
+ pNodesClone.removeAll(feedInfo.computeLocations);
+ pNodesClone.removeAll(feedInfo.ingestLocations);
+ pNodesClone.removeAll(chosenReplacements);
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ for (String candidateNode : pNodesClone) {
+ LOGGER.info("Candidate for replacement:" + candidateNode);
+ }
+ }
+ if (!pNodesClone.isEmpty()) {
+ String[] participantNodesArray = pNodesClone.toArray(new String[] {});
+
+ replacement = participantNodesArray[new Random().nextInt(participantNodesArray.length)];
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Participant Node: " + replacement + " chosen as replacement for " + failedNodeId);
+ }
+ }
+ }
+
+ if (replacement == null) {
+ feedInfo.computeLocations.removeAll(failedNodeIds);
+ boolean computeNodeSubstitute = (feedInfo.computeLocations.size() > 1);
+ if (computeNodeSubstitute) {
+ replacement = feedInfo.computeLocations.get(0);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Compute node:" + replacement + " chosen to replace " + failedNodeId);
+ }
+ } else {
+ replacement = feedInfo.storageLocations.get(0);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Storage node:" + replacement + " chosen to replace " + failedNodeId);
+ }
+ }
+ }
+ return replacement;
+ }
+
+ private void alterFeedJobSpec(FeedInfo feedInfo, AddNodeWorkResponse resp, String failedNodeId, String replacement) {
+ if (replacement == null) {
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("Unable to find replacement for failed node :" + failedNodeId);
+ LOGGER.severe("Feed: " + feedInfo.feedConnectionId + " will be terminated");
+ }
+ List<FeedInfo> feedsToTerminate = new ArrayList<FeedInfo>();
+ feedsToTerminate.add(feedInfo);
+ Thread t = new Thread(new FeedsDeActivator(feedsToTerminate));
+ t.start();
+ } else {
+ replaceNode(feedInfo.jobSpec, failedNodeId, replacement);
+ }
+ }
+
+ private void replaceNode(JobSpecification jobSpec, String failedNodeId, String replacementNode) {
+ Set<Constraint> userConstraints = jobSpec.getUserConstraints();
+ List<Constraint> locationConstraintsToReplace = new ArrayList<Constraint>();
+ List<Constraint> countConstraintsToReplace = new ArrayList<Constraint>();
+ List<OperatorDescriptorId> modifiedOperators = new ArrayList<OperatorDescriptorId>();
+ Map<OperatorDescriptorId, List<Constraint>> candidateConstraints = new HashMap<OperatorDescriptorId, List<Constraint>>();
+ Map<OperatorDescriptorId, Map<Integer, String>> newConstraints = new HashMap<OperatorDescriptorId, Map<Integer, String>>();
+ OperatorDescriptorId opId = null;
+ for (Constraint constraint : userConstraints) {
+ LValueConstraintExpression lexpr = constraint.getLValue();
+ ConstraintExpression cexpr = constraint.getRValue();
+ switch (lexpr.getTag()) {
+ case PARTITION_COUNT:
+ opId = ((PartitionCountExpression) lexpr).getOperatorDescriptorId();
+ if (modifiedOperators.contains(opId)) {
+ countConstraintsToReplace.add(constraint);
+ } else {
+ List<Constraint> clist = candidateConstraints.get(opId);
+ if (clist == null) {
+ clist = new ArrayList<Constraint>();
+ candidateConstraints.put(opId, clist);
+ }
+ clist.add(constraint);
+ }
+ break;
+ case PARTITION_LOCATION:
+ opId = ((PartitionLocationExpression) lexpr).getOperatorDescriptorId();
+ String oldLocation = (String) ((ConstantExpression) cexpr).getValue();
+ if (oldLocation.equals(failedNodeId)) {
+ locationConstraintsToReplace.add(constraint);
+ modifiedOperators.add(((PartitionLocationExpression) lexpr).getOperatorDescriptorId());
+ Map<Integer, String> newLocs = newConstraints.get(opId);
+ if (newLocs == null) {
+ newLocs = new HashMap<Integer, String>();
+ newConstraints.put(opId, newLocs);
+ }
+ int partition = ((PartitionLocationExpression) lexpr).getPartition();
+ newLocs.put(partition, replacementNode);
+ } else {
+ if (modifiedOperators.contains(opId)) {
+ locationConstraintsToReplace.add(constraint);
+ Map<Integer, String> newLocs = newConstraints.get(opId);
+ if (newLocs == null) {
+ newLocs = new HashMap<Integer, String>();
+ newConstraints.put(opId, newLocs);
+ }
+ int partition = ((PartitionLocationExpression) lexpr).getPartition();
+ newLocs.put(partition, oldLocation);
+ } else {
+ List<Constraint> clist = candidateConstraints.get(opId);
+ if (clist == null) {
+ clist = new ArrayList<Constraint>();
+ candidateConstraints.put(opId, clist);
+ }
+ clist.add(constraint);
+ }
+ }
+ break;
+ }
+ }
+
+ jobSpec.getUserConstraints().removeAll(locationConstraintsToReplace);
+ jobSpec.getUserConstraints().removeAll(countConstraintsToReplace);
+
+ for (OperatorDescriptorId mopId : modifiedOperators) {
+ List<Constraint> clist = candidateConstraints.get(mopId);
+ if (clist != null && !clist.isEmpty()) {
+ jobSpec.getUserConstraints().removeAll(clist);
+
+ for (Constraint c : clist) {
+ if (c.getLValue().getTag().equals(ExpressionTag.PARTITION_LOCATION)) {
+ ConstraintExpression cexpr = c.getRValue();
+ int partition = ((PartitionLocationExpression) c.getLValue()).getPartition();
+ String oldLocation = (String) ((ConstantExpression) cexpr).getValue();
+ newConstraints.get(mopId).put(partition, oldLocation);
+ }
+ }
+ }
+ }
+
+ for (Entry<OperatorDescriptorId, Map<Integer, String>> entry : newConstraints.entrySet()) {
+ OperatorDescriptorId nopId = entry.getKey();
+ Map<Integer, String> clist = entry.getValue();
+ IOperatorDescriptor op = jobSpec.getOperatorMap().get(nopId);
+ String[] locations = new String[clist.size()];
+ for (int i = 0; i < locations.length; i++) {
+ locations[i] = clist.get(i);
+ }
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, op, locations);
+ }
+
+ }
+
+ public void registerFeedWork(int workId, FeedFailureReport failureReport) {
+ feedsWaitingForResponse.put(workId, failureReport);
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
index 96d0617..187394a 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
@@ -25,14 +25,18 @@
import edu.uci.ics.asterix.common.api.AsterixThreadFactory;
import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
+import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
import edu.uci.ics.asterix.common.transactions.IRecoveryManager.SystemState;
+import edu.uci.ics.asterix.event.schema.cluster.Cluster;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
import edu.uci.ics.asterix.metadata.MetadataManager;
import edu.uci.ics.asterix.metadata.MetadataNode;
import edu.uci.ics.asterix.metadata.api.IAsterixStateProxy;
import edu.uci.ics.asterix.metadata.api.IMetadataNode;
import edu.uci.ics.asterix.metadata.bootstrap.MetadataBootstrap;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceRepository;
import edu.uci.ics.hyracks.api.application.INCApplicationContext;
import edu.uci.ics.hyracks.api.application.INCApplicationEntryPoint;
@@ -59,14 +63,22 @@
JVMShutdownHook sHook = new JVMShutdownHook(this);
Runtime.getRuntime().addShutdownHook(sHook);
-
runtimeContext = new AsterixAppRuntimeContext(ncApplicationContext);
+ AsterixMetadataProperties metadataProperties = ((IAsterixPropertiesProvider) runtimeContext)
+ .getMetadataProperties();
+ if (!metadataProperties.getNodeNames().contains(ncApplicationContext.getNodeId())) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Substitute node joining : " + ncApplicationContext.getNodeId());
+ }
+ updateOnNodeJoin();
+ }
runtimeContext.initialize();
ncApplicationContext.setApplicationObject(runtimeContext);
// #. recover if the system is corrupted by checking system state.
IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem().getRecoveryManager();
systemState = recoveryMgr.getSystemState();
+
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("System is in a state: " + systemState);
}
@@ -130,21 +142,25 @@
isMetadataNode = nodeId.equals(metadataProperties.getMetadataNodeName());
if (isMetadataNode) {
registerRemoteMetadataNode(proxy);
+ }
+ MetadataManager.INSTANCE = new MetadataManager(proxy, metadataProperties);
+ MetadataManager.INSTANCE.init();
+ if (isMetadataNode) {
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Bootstrapping metadata");
}
- MetadataManager.INSTANCE = new MetadataManager(proxy, metadataProperties);
- MetadataManager.INSTANCE.init();
+
MetadataBootstrap.startUniverse(((IAsterixPropertiesProvider) runtimeContext), ncApplicationContext,
systemState == SystemState.NEW_UNIVERSE);
MetadataBootstrap.startDDLRecovery();
}
+ ExternalLibraryBootstrap.setUpExternaLibraries(isMetadataNode);
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Starting lifecycle components");
}
-
+
Map<String, String> lifecycleMgmtConfiguration = new HashMap<String, String>();
String key = LifeCycleComponentManager.Config.DUMP_PATH_KEY;
String value = metadataProperties.getCoredumpPath(nodeId);
@@ -156,7 +172,7 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Configured:" + LifeCycleComponentManager.INSTANCE);
}
-
+
LifeCycleComponentManager.INSTANCE.startAll();
IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem().getRecoveryManager();
@@ -177,6 +193,47 @@
}
}
+ private void updateOnNodeJoin() {
+ AsterixMetadataProperties metadataProperties = ((IAsterixPropertiesProvider) runtimeContext)
+ .getMetadataProperties();
+ AsterixTransactionProperties txnProperties = ((IAsterixPropertiesProvider) runtimeContext)
+ .getTransactionProperties();
+ if (!metadataProperties.getNodeNames().contains(nodeId)) {
+ metadataProperties.getNodeNames().add(nodeId);
+ Cluster cluster = AsterixClusterProperties.INSTANCE.getCluster();
+ String asterixInstanceName = cluster.getInstanceName();
+ Node self = null;
+ for (Node node : cluster.getSubstituteNodes().getNode()) {
+ String ncId = asterixInstanceName + "_" + node.getId();
+ if (ncId.equalsIgnoreCase(nodeId)) {
+ String storeDir = node.getStore() == null ? cluster.getStore() : node.getStore();
+ metadataProperties.getStores().put(nodeId, storeDir.split(","));
+
+ String coredumpPath = node.getLogDir() == null ? cluster.getLogDir() : node.getLogDir();
+ metadataProperties.getCoredumpPaths().put(nodeId, coredumpPath);
+
+ String txnLogDir = node.getTxnLogDir() == null ? cluster.getTxnLogDir() : node.getTxnLogDir();
+ txnProperties.getLogDirectories().put(nodeId, txnLogDir);
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Store set to : " + storeDir);
+ LOGGER.info("Coredump dir set to : " + coredumpPath);
+ LOGGER.info("Transaction log dir set to :" + txnLogDir);
+ }
+ self = node;
+ break;
+ }
+ }
+ if (self != null) {
+ cluster.getSubstituteNodes().getNode().remove(self);
+ cluster.getNode().add(self);
+ } else {
+ throw new IllegalStateException("Unknown node joining the cluster");
+ }
+ }
+
+ }
+
/**
* Shutdown hook that invokes {@link NCApplicationEntryPoint#stop() stop} method.
*/
@@ -202,4 +259,4 @@
}
}
-}
\ No newline at end of file
+}
diff --git a/asterix-app/src/main/resources/feed/dashboard.html b/asterix-app/src/main/resources/feed/dashboard.html
new file mode 100644
index 0000000..805f8ea
--- /dev/null
+++ b/asterix-app/src/main/resources/feed/dashboard.html
@@ -0,0 +1,127 @@
+<html>
+ <head>
+ <script type="text/javascript" src="http://ajax.googleapis.com/ajax/libs/jquery/1.6.2/jquery.min.js"></script>
+ <script type="text/javascript" src="/webui/static/js/smoothie.js"></script>
+ <script type="text/javascript">
+ $(document).ready(function() {
+ var feedSeries = new TimeSeries();
+ var state = "ACTIVE";
+ var dataverse = "%s";
+ var dataset = "%s";
+ var feed = "%s";
+ var ingestLocations = "%s";
+ var computeLocations = "%s";
+ var storageLocations = "%s";
+ var ingestionPolicy = "%s";
+ var activeSince = "%s";
+ var targetUrl = "/feed/data?dataverse=" + dataverse + "&dataset=" + dataset + "&feed=" + feed;
+
+
+ var ingestionNodes = ingestLocations.split(",");
+ var numIngestionNodes = ingestionNodes.length;
+ var seriesOptions = { strokeStyle: 'rgba(0, 255, 0, 1)', fillStyle: 'rgba(0, 255, 0, 0.2)', lineWidth: 4 };
+ var ingestionTimeSeries = new Array();
+ var graphNames = new Array();
+
+ $.ajaxSetup({ cache: false });
+ setInterval(fetchFeedReport, 5000);
+ function fetchFeedReport() {
+ $.ajax({
+ url: '/feed/data?dataverse=' + dataverse + '&dataset=' + dataset + '&feed=' + feed,
+ method: 'GET',
+ dataType: 'json',
+ success: onFeedReportReceived
+ });
+ }
+
+
+ function onFeedReportReceived(data) {
+ var status = data["status"];
+ if(status == ("ended")){
+ ingestLocations = " ";
+ computeLocations = " ";
+ storageLocations = " ";
+ ingestionPolicy = " ";
+ state = "INACTIVE";
+ document.location.reload(true);
+ } else {
+ var type = data["type"];
+ if (type == ("reload")) {
+ ingestLocations = data["ingestLocations"];
+ computeLocations = data["computeLocations"];
+ storageLocations = data["storageLocations"];
+ document.location.reload(true);
+ } else {
+ var report = data["value"];
+ var tputArray = report.split("|");
+ var covered = 0;
+ var totalTput = 0;
+ for( var i = 0; i < tputArray.length-1; i ++){
+ ingestionTimeSeries[i].append(data["time"], tputArray[i]);
+ covered++;
+ totalTput += parseInt(tputArray[i]);
+ }
+ for( var j = covered; j < numIngestionNodes; j++){
+ ingestionTimeSeries[j].append(data["time"], 0);
+ }
+ ingestionTimeSeries[numIngestionNodes].append(data["time"], totalTput);
+ }
+ }
+ }
+
+ function myYRangeFunction(range) {
+ var min = 0;
+ var max = 5000;
+ return {min: min, max: max};
+ }
+
+ function initTimeline(ingestLocations) {
+
+ document.write("<i>" + "Feed Ingestion" + " " + "<i>");
+ document.write("<br />" + "Ingestion Locations: " + ingestLocations.replace(",",", "));
+ document.write("<br />" + "Compute Locations: " + computeLocations.replace(",",", "));
+ document.write("<br />" + "Storage Locations: " + storageLocations.replace(",",", "));
+ document.write("<br />" + "Ingestion Policy: " + ingestionPolicy);
+ document.write("<br />" + "Status: " + state);
+ document.write("<br />");
+ document.write("<br />");
+
+ for( var i = 0; i < numIngestionNodes; i++){
+ graphNames[i] = "Partition " + i;
+ }
+
+ if(numIngestionNodes > 1){
+ graphNames[numIngestionNodes] = "IngestionThroughput";
+ drawCanvas(graphNames[numIngestionNodes]);
+ ingestionTimeSeries[numIngestionNodes] = new TimeSeries();
+ drawChart(graphNames[numIngestionNodes], ingestionTimeSeries[numIngestionNodes]);
+ }
+
+ for( var j = 0; j < numIngestionNodes; j++){
+ drawCanvas(graphNames[j]);
+ ingestionTimeSeries[j] = new TimeSeries();
+ drawChart(graphNames[j], ingestionTimeSeries[j]);
+ }
+ }
+
+ function drawCanvas(chartName) {
+ document.write("<br />");
+ document.write("<br />");
+ document.write("<i>" + chartName + "</i>");
+ document.write("<br />");
+ document.write("<canvas id="+ "\"" + chartName + "\"" + " " + "width=\"500\" height=\"250\"></canvas>");
+ }
+
+ function drawChart(chartName, ingestionTimeSeries) {
+ var ingestionChart = new SmoothieChart({ timestampFormatter:SmoothieChart.timeFormatter, interpolation:'linear', minValue:0, millisPerPixel: 20, grid: { strokeStyle: '#555555', lineWidth: 1, millisPerLine: 1000, verticalSections: 10 }});
+ ingestionChart.addTimeSeries(ingestionTimeSeries, seriesOptions);
+ ingestionChart.streamTo(document.getElementById(chartName, 500));
+ }
+
+ initTimeline(ingestLocations);
+ });
+ </script>
+ </head>
+ <body></body>
+</html>
+
diff --git a/asterix-app/src/main/resources/feed/home.html b/asterix-app/src/main/resources/feed/home.html
new file mode 100644
index 0000000..5b1721b
--- /dev/null
+++ b/asterix-app/src/main/resources/feed/home.html
@@ -0,0 +1,82 @@
+<!--
+ ! Copyright 2009-2013 by The Regents of the University of California
+ ! Licensed under the Apache License, Version 2.0 (the "License");
+ ! you may not use this file except in compliance with the License.
+ ! you may obtain a copy of the License from
+ !
+ ! http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing, software
+ ! distributed under the License is distributed on an "AS IS" BASIS,
+ ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ ! See the License for the specific language governing permissions and
+ ! limitations under the License.
+ !-->
+<!DOCTYPE html>
+<html lang="en">
+<head>
+<meta name="description" content="ASTERIX WEB PAGE" />
+<meta name="viewport" content="width=device-width, initial-scale=1.0">
+<link href='http://fonts.googleapis.com/css?family=Bitter|PT+Sans+Caption|Open+Sans' rel='stylesheet' type='text/css'>
+<script src="/webui/static/js/jquery.min.js"></script>
+
+<link href="/webui/static/css/bootstrap.min.css" rel="stylesheet" type="text/css" />
+<link href="/webui/static/css/bootstrap-responsive.min.css" rel="stylesheet" type="text/css" />
+
+<script src="/webui/static/js/bootstrap.min.js"></script>
+
+<link href="/webui/static/css/style.css" rel="stylesheet" type="text/css" />
+
+
+<meta charset=utf-8 />
+<title>AsterixDB Web Interface</title>
+</head>
+
+<body>
+ <div class="navbar navbar-fixed-top">
+ <div class="navbar-inner">
+ <div class="container">
+ <a class="btn btn-navbar" data-toggle="collapse" data-target=".nav-collapse">
+ <span class="icon-bar"></span>
+ <span class="icon-bar"></span>
+ <span class="icon-bar"></span>
+ </a>
+
+ <!-- Temporary logo placeholder -->
+ <a class="brand" href="#"><img src="/webui/static/img/finalasterixlogo.png"></a>
+
+ <div class="nav-collapse collapse">
+ <ul class="nav">
+ <li><a href="http://code.google.com/p/asterixdb/" target="_blank">
+ Open source<img class="extarget" src="/webui/static/img/targetlink.png"/></a></li>
+ <li><a href="http://code.google.com/p/asterixdb/issues/list" target="_blank">
+ File issues<img class="extarget" src="/webui/static/img/targetlink.png"/></a></li>
+ <li><a href="https://groups.google.com/forum/?fromgroups#!forum/asterixdb-users" target="_blank">
+ Contact<img class="extarget" src="/webui/static/img/targetlink.png"/></a></li>
+ </ul>
+ </div><!--/.nav-collapse -->
+ </div>
+ </div>
+ </div>
+
+ <div class="content">
+ <div class="container">
+ <div class="row-fluid">
+ <div class="span12">
+ %s
+ </div>
+ </div>
+ </div>
+ </div>
+ <div class="footer">
+ <section class="line"><hr></section>
+ <section class="content">
+ <section class="left">
+ </section>
+ <section class="right">
+ © Copyright 2013 University of California, Irvine
+ </section>
+ </section>
+ </div>
+</body>
+</html>
diff --git a/asterix-app/src/main/resources/feed/smoothie.js b/asterix-app/src/main/resources/feed/smoothie.js
new file mode 100644
index 0000000..4e46fa7
--- /dev/null
+++ b/asterix-app/src/main/resources/feed/smoothie.js
@@ -0,0 +1,660 @@
+// MIT License:
+//
+// Copyright (c) 2010-2013, Joe Walnes
+//
+// Permission is hereby granted, free of charge, to any person obtaining a copy
+// of this software and associated documentation files (the "Software"), to deal
+// in the Software without restriction, including without limitation the rights
+// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+// copies of the Software, and to permit persons to whom the Software is
+// furnished to do so, subject to the following conditions:
+//
+// The above copyright notice and this permission notice shall be included in
+// all copies or substantial portions of the Software.
+//
+// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+// THE SOFTWARE.
+
+/**
+ * Smoothie Charts - http://smoothiecharts.org/
+ * (c) 2010-2013, Joe Walnes
+ * 2013, Drew Noakes
+ *
+ * v1.0: Main charting library, by Joe Walnes
+ * v1.1: Auto scaling of axis, by Neil Dunn
+ * v1.2: fps (frames per second) option, by Mathias Petterson
+ * v1.3: Fix for divide by zero, by Paul Nikitochkin
+ * v1.4: Set minimum, top-scale padding, remove timeseries, add optional timer to reset bounds, by Kelley Reynolds
+ * v1.5: Set default frames per second to 50... smoother.
+ * .start(), .stop() methods for conserving CPU, by Dmitry Vyal
+ * options.interpolation = 'bezier' or 'line', by Dmitry Vyal
+ * options.maxValue to fix scale, by Dmitry Vyal
+ * v1.6: minValue/maxValue will always get converted to floats, by Przemek Matylla
+ * v1.7: options.grid.fillStyle may be a transparent color, by Dmitry A. Shashkin
+ * Smooth rescaling, by Kostas Michalopoulos
+ * v1.8: Set max length to customize number of live points in the dataset with options.maxDataSetLength, by Krishna Narni
+ * v1.9: Display timestamps along the bottom, by Nick and Stev-io
+ * (https://groups.google.com/forum/?fromgroups#!topic/smoothie-charts/-Ywse8FCpKI%5B1-25%5D)
+ * Refactored by Krishna Narni, to support timestamp formatting function
+ * v1.10: Switch to requestAnimationFrame, removed the now obsoleted options.fps, by Gergely Imreh
+ * v1.11: options.grid.sharpLines option added, by @drewnoakes
+ * Addressed warning seen in Firefox when seriesOption.fillStyle undefined, by @drewnoakes
+ * v1.12: Support for horizontalLines added, by @drewnoakes
+ * Support for yRangeFunction callback added, by @drewnoakes
+ * v1.13: Fixed typo (#32), by @alnikitich
+ * v1.14: Timer cleared when last TimeSeries removed (#23), by @davidgaleano
+ * Fixed diagonal line on chart at start/end of data stream, by @drewnoakes
+ * v1.15: Support for npm package (#18), by @dominictarr
+ * Fixed broken removeTimeSeries function (#24) by @davidgaleano
+ * Minor performance and tidying, by @drewnoakes
+ * v1.16: Bug fix introduced in v1.14 relating to timer creation/clearance (#23), by @drewnoakes
+ * TimeSeries.append now deals with out-of-order timestamps, and can merge duplicates, by @zacwitte (#12)
+ * Documentation and some local variable renaming for clarity, by @drewnoakes
+ * v1.17: Allow control over font size (#10), by @drewnoakes
+ * Timestamp text won't overlap, by @drewnoakes
+ * v1.18: Allow control of max/min label precision, by @drewnoakes
+ * Added 'borderVisible' chart option, by @drewnoakes
+ * Allow drawing series with fill but no stroke (line), by @drewnoakes
+ */
+
+;(function(exports) {
+
+ var Util = {
+ extend: function() {
+ arguments[0] = arguments[0] || {};
+ for (var i = 1; i < arguments.length; i++)
+ {
+ for (var key in arguments[i])
+ {
+ if (arguments[i].hasOwnProperty(key))
+ {
+ if (typeof(arguments[i][key]) === 'object') {
+ if (arguments[i][key] instanceof Array) {
+ arguments[0][key] = arguments[i][key];
+ } else {
+ arguments[0][key] = Util.extend(arguments[0][key], arguments[i][key]);
+ }
+ } else {
+ arguments[0][key] = arguments[i][key];
+ }
+ }
+ }
+ }
+ return arguments[0];
+ }
+ };
+
+ /**
+ * Initialises a new <code>TimeSeries</code> with optional data options.
+ *
+ * Options are of the form (defaults shown):
+ *
+ * <pre>
+ * {
+ * resetBounds: true, // enables/disables automatic scaling of the y-axis
+ * resetBoundsInterval: 3000 // the period between scaling calculations, in millis
+ * }
+ * </pre>
+ *
+ * Presentation options for TimeSeries are specified as an argument to <code>SmoothieChart.addTimeSeries</code>.
+ *
+ * @constructor
+ */
+ function TimeSeries(options) {
+ this.options = Util.extend({}, TimeSeries.defaultOptions, options);
+ this.data = [];
+ this.maxValue = Number.NaN; // The maximum value ever seen in this TimeSeries.
+ this.minValue = Number.NaN; // The minimum value ever seen in this TimeSeries.
+ }
+
+ TimeSeries.defaultOptions = {
+ resetBoundsInterval: 3000,
+ resetBounds: true
+ };
+
+ /**
+ * Recalculate the min/max values for this <code>TimeSeries</code> object.
+ *
+ * This causes the graph to scale itself in the y-axis.
+ */
+ TimeSeries.prototype.resetBounds = function() {
+ if (this.data.length) {
+ // Walk through all data points, finding the min/max value
+ this.maxValue = this.data[0][1];
+ this.minValue = this.data[0][1];
+ for (var i = 1; i < this.data.length; i++) {
+ var value = this.data[i][1];
+ if (value > this.maxValue) {
+ this.maxValue = value;
+ }
+ if (value < this.minValue) {
+ this.minValue = value;
+ }
+ }
+ } else {
+ // No data exists, so set min/max to NaN
+ this.maxValue = Number.NaN;
+ this.minValue = Number.NaN;
+ }
+ };
+
+ /**
+ * Adds a new data point to the <code>TimeSeries</code>, preserving chronological order.
+ *
+ * @param timestamp the position, in time, of this data point
+ * @param value the value of this data point
+ * @param sumRepeatedTimeStampValues if <code>timestamp</code> has an exact match in the series, this flag controls
+ * whether it is replaced, or the values summed (defaults to false.)
+ */
+ TimeSeries.prototype.append = function(timestamp, value, sumRepeatedTimeStampValues) {
+ // Rewind until we hit an older timestamp
+ var i = this.data.length - 1;
+ while (i > 0 && this.data[i][0] > timestamp) {
+ i--;
+ }
+
+ if (this.data.length > 0 && this.data[i][0] === timestamp) {
+ // Update existing values in the array
+ if (sumRepeatedTimeStampValues) {
+ // Sum this value into the existing 'bucket'
+ this.data[i][1] += value;
+ value = this.data[i][1];
+ } else {
+ // Replace the previous value
+ this.data[i][1] = value;
+ }
+ } else if (i < this.data.length - 1) {
+ // Splice into the correct position to keep timestamps in order
+ this.data.splice(i + 1, 0, [timestamp, value]);
+ } else {
+ // Add to the end of the array
+ this.data.push([timestamp, value]);
+ }
+
+ this.maxValue = isNaN(this.maxValue) ? value : Math.max(this.maxValue, value);
+ this.minValue = isNaN(this.minValue) ? value : Math.min(this.minValue, value);
+ };
+
+ TimeSeries.prototype.dropOldData = function(oldestValidTime, maxDataSetLength) {
+ // We must always keep one expired data point as we need this to draw the
+ // line that comes into the chart from the left, but any points prior to that can be removed.
+ var removeCount = 0;
+ while (this.data.length - removeCount >= maxDataSetLength && this.data[removeCount + 1][0] < oldestValidTime) {
+ removeCount++;
+ }
+ if (removeCount !== 0) {
+ this.data.splice(0, removeCount);
+ }
+ };
+
+ /**
+ * Initialises a new <code>SmoothieChart</code>.
+ *
+ * Options are optional, and should be of the form below. Just specify the values you
+ * need and the rest will be given sensible defaults as shown:
+ *
+ * <pre>
+ * {
+ * minValue: undefined, // specify to clamp the lower y-axis to a given value
+ * maxValue: undefined, // specify to clamp the upper y-axis to a given value
+ * maxValueScale: 1, // allows proportional padding to be added above the chart. for 10% padding, specify 1.1.
+ * yRangeFunction: undefined, // function({min: , max: }) { return {min: , max: }; }
+ * scaleSmoothing: 0.125, // controls the rate at which y-value zoom animation occurs
+ * millisPerPixel: 20, // sets the speed at which the chart pans by
+ * maxDataSetLength: 2,
+ * interpolation: 'bezier' // or 'linear'
+ * timestampFormatter: null, // Optional function to format time stamps for bottom of chart. You may use SmoothieChart.timeFormatter, or your own: function(date) { return ''; }
+ * horizontalLines: [], // [ { value: 0, color: '#ffffff', lineWidth: 1 } ],
+ * grid:
+ * {
+ * fillStyle: '#000000', // the background colour of the chart
+ * lineWidth: 1, // the pixel width of grid lines
+ * strokeStyle: '#777777', // colour of grid lines
+ * millisPerLine: 1000, // distance between vertical grid lines
+ * sharpLines: false, // controls whether grid lines are 1px sharp, or softened
+ * verticalSections: 2, // number of vertical sections marked out by horizontal grid lines
+ * borderVisible: true // whether the grid lines trace the border of the chart or not
+ * },
+ * labels
+ * {
+ * disabled: false, // enables/disables labels showing the min/max values
+ * fillStyle: '#ffffff', // colour for text of labels,
+ * fontSize: 15,
+ * fontFamily: 'sans-serif',
+ * precision: 2
+ * },
+ * }
+ * </pre>
+ *
+ * @constructor
+ */
+ function SmoothieChart(options) {
+ this.options = Util.extend({}, SmoothieChart.defaultChartOptions, options);
+ this.seriesSet = [];
+ this.currentValueRange = 1;
+ this.currentVisMinValue = 0;
+ }
+
+ SmoothieChart.defaultChartOptions = {
+ millisPerPixel: 20,
+ maxValueScale: 1,
+ interpolation: 'bezier',
+ scaleSmoothing: 0.125,
+ maxDataSetLength: 2,
+ grid: {
+ fillStyle: '#000000',
+ strokeStyle: '#777777',
+ lineWidth: 1,
+ sharpLines: false,
+ millisPerLine: 1000,
+ verticalSections: 2,
+ borderVisible: true
+ },
+ labels: {
+ fillStyle: '#ffffff',
+ disabled: false,
+ fontSize: 10,
+ fontFamily: 'monospace',
+ precision: 2
+ },
+ horizontalLines: []
+ };
+
+ // Based on http://inspirit.github.com/jsfeat/js/compatibility.js
+ SmoothieChart.AnimateCompatibility = (function() {
+ // TODO this global variable will cause bugs if more than one chart is used and the browser does not support *requestAnimationFrame natively
+ var lastTime = 0,
+ requestAnimationFrame = function(callback, element) {
+ var requestAnimationFrame =
+ window.requestAnimationFrame ||
+ window.webkitRequestAnimationFrame ||
+ window.mozRequestAnimationFrame ||
+ window.oRequestAnimationFrame ||
+ window.msRequestAnimationFrame ||
+ function(callback) {
+ var currTime = new Date().getTime(),
+ timeToCall = Math.max(0, 16 - (currTime - lastTime)),
+ id = window.setTimeout(function() {
+ callback(currTime + timeToCall);
+ }, timeToCall);
+ lastTime = currTime + timeToCall;
+ return id;
+ };
+ return requestAnimationFrame.call(window, callback, element);
+ },
+ cancelAnimationFrame = function(id) {
+ var cancelAnimationFrame =
+ window.cancelAnimationFrame ||
+ function(id) {
+ clearTimeout(id);
+ };
+ return cancelAnimationFrame.call(window, id);
+ };
+
+ return {
+ requestAnimationFrame: requestAnimationFrame,
+ cancelAnimationFrame: cancelAnimationFrame
+ };
+ })();
+
+ SmoothieChart.defaultSeriesPresentationOptions = {
+ lineWidth: 1,
+ strokeStyle: '#ffffff'
+ };
+
+ /**
+ * Adds a <code>TimeSeries</code> to this chart, with optional presentation options.
+ *
+ * Presentation options should be of the form (defaults shown):
+ *
+ * <pre>
+ * {
+ * lineWidth: 1,
+ * strokeStyle: '#ffffff',
+ * fillStyle: undefined
+ * }
+ * </pre>
+ */
+ SmoothieChart.prototype.addTimeSeries = function(timeSeries, options) {
+ this.seriesSet.push({timeSeries: timeSeries, options: Util.extend({}, SmoothieChart.defaultSeriesPresentationOptions, options)});
+ if (timeSeries.options.resetBounds && timeSeries.options.resetBoundsInterval > 0) {
+ timeSeries.resetBoundsTimerId = setInterval(
+ function() {
+ timeSeries.resetBounds();
+ },
+ timeSeries.options.resetBoundsInterval
+ );
+ }
+ };
+
+ /**
+ * Removes the specified <code>TimeSeries</code> from the chart.
+ */
+ SmoothieChart.prototype.removeTimeSeries = function(timeSeries) {
+ // Find the correct timeseries to remove, and remove it
+ var numSeries = this.seriesSet.length;
+ for (var i = 0; i < numSeries; i++) {
+ if (this.seriesSet[i].timeSeries === timeSeries) {
+ this.seriesSet.splice(i, 1);
+ break;
+ }
+ }
+ // If a timer was operating for that timeseries, remove it
+ if (timeSeries.resetBoundsTimerId) {
+ // Stop resetting the bounds, if we were
+ clearInterval(timeSeries.resetBoundsTimerId);
+ }
+ };
+
+ /**
+ * Instructs the <code>SmoothieChart</code> to start rendering to the provided canvas, with specified delay.
+ *
+ * @param canvas the target canvas element
+ * @param delayMillis an amount of time to wait before a data point is shown. This can prevent the end of the series
+ * from appearing on screen, with new values flashing into view, at the expense of some latency.
+ */
+ SmoothieChart.prototype.streamTo = function(canvas, delayMillis) {
+ this.canvas = canvas;
+ this.delay = delayMillis;
+ this.start();
+ };
+
+ /**
+ * Starts the animation of this chart.
+ */
+ SmoothieChart.prototype.start = function() {
+ if (this.frame) {
+ // We're already running, so just return
+ return;
+ }
+
+ // Renders a frame, and queues the next frame for later rendering
+ var animate = function() {
+ this.frame = SmoothieChart.AnimateCompatibility.requestAnimationFrame(function() {
+ this.render();
+ animate();
+ }.bind(this));
+ }.bind(this);
+
+ animate();
+ };
+
+ /**
+ * Stops the animation of this chart.
+ */
+ SmoothieChart.prototype.stop = function() {
+ if (this.frame) {
+ SmoothieChart.AnimateCompatibility.cancelAnimationFrame(this.frame);
+ delete this.frame;
+ }
+ };
+
+ SmoothieChart.prototype.updateValueRange = function() {
+ // Calculate the current scale of the chart, from all time series.
+ var chartOptions = this.options,
+ chartMaxValue = Number.NaN,
+ chartMinValue = Number.NaN;
+
+ for (var d = 0; d < this.seriesSet.length; d++) {
+ // TODO(ndunn): We could calculate / track these values as they stream in.
+ var timeSeries = this.seriesSet[d].timeSeries;
+ if (!isNaN(timeSeries.maxValue)) {
+ chartMaxValue = !isNaN(chartMaxValue) ? Math.max(chartMaxValue, timeSeries.maxValue) : timeSeries.maxValue;
+ }
+
+ if (!isNaN(timeSeries.minValue)) {
+ chartMinValue = !isNaN(chartMinValue) ? Math.min(chartMinValue, timeSeries.minValue) : timeSeries.minValue;
+ }
+ }
+
+ // Scale the chartMaxValue to add padding at the top if required
+ if (chartOptions.maxValue != null) {
+ chartMaxValue = chartOptions.maxValue;
+ } else {
+ chartMaxValue *= chartOptions.maxValueScale;
+ }
+
+ // Set the minimum if we've specified one
+ if (chartOptions.minValue != null) {
+ chartMinValue = chartOptions.minValue;
+ }
+
+ // If a custom range function is set, call it
+ if (this.options.yRangeFunction) {
+ var range = this.options.yRangeFunction({min: chartMinValue, max: chartMaxValue});
+ chartMinValue = range.min;
+ chartMaxValue = range.max;
+ }
+
+ if (!isNaN(chartMaxValue) && !isNaN(chartMinValue)) {
+ var targetValueRange = chartMaxValue - chartMinValue;
+ this.currentValueRange += chartOptions.scaleSmoothing * (targetValueRange - this.currentValueRange);
+ this.currentVisMinValue += chartOptions.scaleSmoothing * (chartMinValue - this.currentVisMinValue);
+ }
+
+ this.valueRange = { min: chartMinValue, max: chartMaxValue };
+ };
+
+ SmoothieChart.prototype.render = function(canvas, time) {
+ canvas = canvas || this.canvas;
+ time = time || new Date().getTime() - (this.delay || 0);
+
+ // TODO only render if the chart has moved at least 1px since the last rendered frame
+
+ // Round time down to pixel granularity, so motion appears smoother.
+ time -= time % this.options.millisPerPixel;
+
+ var context = canvas.getContext('2d'),
+ chartOptions = this.options,
+ dimensions = { top: 0, left: 0, width: canvas.clientWidth, height: canvas.clientHeight },
+ // Calculate the threshold time for the oldest data points.
+ oldestValidTime = time - (dimensions.width * chartOptions.millisPerPixel),
+ valueToYPixel = function(value) {
+ var offset = value - this.currentVisMinValue;
+ return this.currentValueRange === 0
+ ? dimensions.height
+ : dimensions.height - (Math.round((offset / this.currentValueRange) * dimensions.height));
+ }.bind(this),
+ timeToXPixel = function(t) {
+ return Math.round(dimensions.width - ((time - t) / chartOptions.millisPerPixel));
+ };
+
+ this.updateValueRange();
+
+ context.font = chartOptions.labels.fontSize + 'px ' + chartOptions.labels.fontFamily;
+
+ // Save the state of the canvas context, any transformations applied in this method
+ // will get removed from the stack at the end of this method when .restore() is called.
+ context.save();
+
+ // Move the origin.
+ context.translate(dimensions.left, dimensions.top);
+
+ // Create a clipped rectangle - anything we draw will be constrained to this rectangle.
+ // This prevents the occasional pixels from curves near the edges overrunning and creating
+ // screen cheese (that phrase should need no explanation).
+ context.beginPath();
+ context.rect(0, 0, dimensions.width, dimensions.height);
+ context.clip();
+
+ // Clear the working area.
+ context.save();
+ context.fillStyle = chartOptions.grid.fillStyle;
+ context.clearRect(0, 0, dimensions.width, dimensions.height);
+ context.fillRect(0, 0, dimensions.width, dimensions.height);
+ context.restore();
+
+ // Grid lines...
+ context.save();
+ context.lineWidth = chartOptions.grid.lineWidth;
+ context.strokeStyle = chartOptions.grid.strokeStyle;
+ // Vertical (time) dividers.
+ if (chartOptions.grid.millisPerLine > 0) {
+ var textUntilX = dimensions.width - context.measureText(minValueString).width + 4;
+ for (var t = time - (time % chartOptions.grid.millisPerLine);
+ t >= oldestValidTime;
+ t -= chartOptions.grid.millisPerLine) {
+ var gx = timeToXPixel(t);
+ if (chartOptions.grid.sharpLines) {
+ gx -= 0.5;
+ }
+ context.beginPath();
+ context.moveTo(gx, 0);
+ context.lineTo(gx, dimensions.height);
+ context.stroke();
+ context.closePath();
+
+ // Display timestamp at bottom of this line if requested, and it won't overlap
+ if (chartOptions.timestampFormatter && gx < textUntilX) {
+ // Formats the timestamp based on user specified formatting function
+ // SmoothieChart.timeFormatter function above is one such formatting option
+ var tx = new Date(t),
+ ts = chartOptions.timestampFormatter(tx),
+ tsWidth = context.measureText(ts).width;
+ textUntilX = gx - tsWidth - 2;
+ context.fillStyle = chartOptions.labels.fillStyle;
+ context.fillText(ts, gx - tsWidth, dimensions.height - 2);
+ }
+ }
+ }
+
+ // Horizontal (value) dividers.
+ for (var v = 1; v < chartOptions.grid.verticalSections; v++) {
+ var gy = Math.round(v * dimensions.height / chartOptions.grid.verticalSections);
+ if (chartOptions.grid.sharpLines) {
+ gy -= 0.5;
+ }
+ context.beginPath();
+ context.moveTo(0, gy);
+ context.lineTo(dimensions.width, gy);
+ context.stroke();
+ context.closePath();
+ }
+ // Bounding rectangle.
+ if (chartOptions.grid.borderVisible) {
+ context.beginPath();
+ context.strokeRect(0, 0, dimensions.width, dimensions.height);
+ context.closePath();
+ }
+ context.restore();
+
+ // Draw any horizontal lines...
+ if (chartOptions.horizontalLines && chartOptions.horizontalLines.length) {
+ for (var hl = 0; hl < chartOptions.horizontalLines.length; hl++) {
+ var line = chartOptions.horizontalLines[hl],
+ hly = Math.round(valueToYPixel(line.value)) - 0.5;
+ context.strokeStyle = line.color || '#ffffff';
+ context.lineWidth = line.lineWidth || 1;
+ context.beginPath();
+ context.moveTo(0, hly);
+ context.lineTo(dimensions.width, hly);
+ context.stroke();
+ context.closePath();
+ }
+ }
+
+ // For each data set...
+ for (var d = 0; d < this.seriesSet.length; d++) {
+ context.save();
+ var timeSeries = this.seriesSet[d].timeSeries,
+ dataSet = timeSeries.data,
+ seriesOptions = this.seriesSet[d].options;
+
+ // Delete old data that's moved off the left of the chart.
+ timeSeries.dropOldData(oldestValidTime, chartOptions.maxDataSetLength);
+
+ // Set style for this dataSet.
+ context.lineWidth = seriesOptions.lineWidth;
+ context.strokeStyle = seriesOptions.strokeStyle;
+ // Draw the line...
+ context.beginPath();
+ // Retain lastX, lastY for calculating the control points of bezier curves.
+ var firstX = 0, lastX = 0, lastY = 0;
+ for (var i = 0; i < dataSet.length && dataSet.length !== 1; i++) {
+ var x = timeToXPixel(dataSet[i][0]),
+ y = valueToYPixel(dataSet[i][1]);
+
+ if (i === 0) {
+ firstX = x;
+ context.moveTo(x, y);
+ } else {
+ switch (chartOptions.interpolation) {
+ case "linear":
+ case "line": {
+ context.lineTo(x,y);
+ break;
+ }
+ case "bezier":
+ default: {
+ // Great explanation of Bezier curves: http://en.wikipedia.org/wiki/Bezier_curve#Quadratic_curves
+ //
+ // Assuming A was the last point in the line plotted and B is the new point,
+ // we draw a curve with control points P and Q as below.
+ //
+ // A---P
+ // |
+ // |
+ // |
+ // Q---B
+ //
+ // Importantly, A and P are at the same y coordinate, as are B and Q. This is
+ // so adjacent curves appear to flow as one.
+ //
+ context.bezierCurveTo( // startPoint (A) is implicit from last iteration of loop
+ Math.round((lastX + x) / 2), lastY, // controlPoint1 (P)
+ Math.round((lastX + x)) / 2, y, // controlPoint2 (Q)
+ x, y); // endPoint (B)
+ break;
+ }
+ }
+ }
+
+ lastX = x; lastY = y;
+ }
+
+ if (dataSet.length > 1) {
+ if (seriesOptions.fillStyle) {
+ // Close up the fill region.
+ context.lineTo(dimensions.width + seriesOptions.lineWidth + 1, lastY);
+ context.lineTo(dimensions.width + seriesOptions.lineWidth + 1, dimensions.height + seriesOptions.lineWidth + 1);
+ context.lineTo(firstX, dimensions.height + seriesOptions.lineWidth);
+ context.fillStyle = seriesOptions.fillStyle;
+ context.fill();
+ }
+
+ if (seriesOptions.strokeStyle && seriesOptions.strokeStyle !== 'none') {
+ context.stroke();
+ }
+ context.closePath();
+ }
+ context.restore();
+ }
+
+ // Draw the axis values on the chart.
+ if (!chartOptions.labels.disabled && !isNaN(this.valueRange.min) && !isNaN(this.valueRange.max)) {
+ var maxValueString = parseFloat(this.valueRange.max).toFixed(chartOptions.labels.precision),
+ minValueString = parseFloat(this.valueRange.min).toFixed(chartOptions.labels.precision);
+ context.fillStyle = chartOptions.labels.fillStyle;
+ context.fillText(maxValueString, dimensions.width - context.measureText(maxValueString).width - 2, chartOptions.labels.fontSize);
+ context.fillText(minValueString, dimensions.width - context.measureText(minValueString).width - 2, dimensions.height - 2);
+ }
+
+ context.restore(); // See .save() above.
+ };
+
+ // Sample timestamp formatting function
+ SmoothieChart.timeFormatter = function(date) {
+ function pad2(number) { return (number < 10 ? '0' : '') + number }
+ return pad2(date.getHours()) + ':' + pad2(date.getMinutes()) + ':' + pad2(date.getSeconds());
+ };
+
+ exports.TimeSeries = TimeSeries;
+ exports.SmoothieChart = SmoothieChart;
+
+})(typeof exports === 'undefined' ? this : exports);
+
diff --git a/asterix-app/src/main/resources/test.properties b/asterix-app/src/main/resources/test.properties
old mode 100755
new mode 100644
diff --git a/asterix-app/src/main/resources/webui/static/js/jquery.autosize-min.js b/asterix-app/src/main/resources/webui/static/js/jquery.autosize-min.js
index b4303a6..2cff1f8 100644
--- a/asterix-app/src/main/resources/webui/static/js/jquery.autosize-min.js
+++ b/asterix-app/src/main/resources/webui/static/js/jquery.autosize-min.js
@@ -1,7 +1 @@
-/*!
- jQuery Autosize v1.16.7
- (c) 2013 Jack Moore - jacklmoore.com
- updated: 2013-03-20
- license: http://www.opensource.org/licenses/mit-license.php
-*/
-(function(e){var t,o,n={className:"autosizejs",append:"",callback:!1},i="hidden",s="border-box",a="lineHeight",l='<textarea tabindex="-1" style="position:absolute; top:-999px; left:0; right:auto; bottom:auto; border:0; -moz-box-sizing:content-box; -webkit-box-sizing:content-box; box-sizing:content-box; word-wrap:break-word; height:0 !important; min-height:0 !important; overflow:hidden;"/>',r=["fontFamily","fontSize","fontWeight","fontStyle","letterSpacing","textTransform","wordSpacing","textIndent"],c="oninput",h="onpropertychange",p=e(l).data("autosize",!0)[0];p.style.lineHeight="99px","99px"===e(p).css(a)&&r.push(a),p.style.lineHeight="",e.fn.autosize=function(a){return a=e.extend({},n,a||{}),p.parentNode!==document.body&&(e(document.body).append(p),p.value="\n\n\n",p.scrollTop=9e4,t=p.scrollHeight===p.scrollTop+p.clientHeight),this.each(function(){function n(){o=b,p.className=a.className,e.each(r,function(e,t){p.style[t]=f.css(t)})}function l(){var e,s,l;if(o!==b&&n(),!d){d=!0,p.value=b.value+a.append,p.style.overflowY=b.style.overflowY,l=parseInt(b.style.height,10),p.style.width=Math.max(f.width(),0)+"px",t?e=p.scrollHeight:(p.scrollTop=0,p.scrollTop=9e4,e=p.scrollTop);var r=parseInt(f.css("maxHeight"),10);r=r&&r>0?r:9e4,e>r?(e=r,s="scroll"):u>e&&(e=u),e+=x,b.style.overflowY=s||i,l!==e&&(b.style.height=e+"px",w&&a.callback.call(b)),setTimeout(function(){d=!1},1)}}var u,d,g,b=this,f=e(b),x=0,w=e.isFunction(a.callback);f.data("autosize")||((f.css("box-sizing")===s||f.css("-moz-box-sizing")===s||f.css("-webkit-box-sizing")===s)&&(x=f.outerHeight()-f.height()),u=Math.max(parseInt(f.css("minHeight"),10)-x,f.height()),g="none"===f.css("resize")||"vertical"===f.css("resize")?"none":"horizontal",f.css({overflow:i,overflowY:i,wordWrap:"break-word",resize:g}).data("autosize",!0),h in b?c in b?b[c]=b.onkeyup=l:b[h]=l:b[c]=l,e(window).on("resize",function(){d=!1,l()}),f.on("autosize",function(){d=!1,l()}),l())})}})(window.jQuery||window.Zepto);
\ No newline at end of file
+(function(e){var t,o,n={className:"autosizejs",append:"",callback:!1},i="hidden",s="border-box",a="lineHeight",l='<textarea tabindex="-1" style="position:absolute; top:-999px; left:0; right:auto; bottom:auto; border:0; -moz-box-sizing:content-box; -webkit-box-sizing:content-box; box-sizing:content-box; word-wrap:break-word; height:0 !important; min-height:0 !important; overflow:hidden;"/>',r=["fontFamily","fontSize","fontWeight","fontStyle","letterSpacing","textTransform","wordSpacing","textIndent"],c="oninput",h="onpropertychange",p=e(l).data("autosize",!0)[0];p.style.lineHeight="99px","99px"===e(p).css(a)&&r.push(a),p.style.lineHeight="",e.fn.autosize=function(a){return a=e.extend({},n,a||{}),p.parentNode!==document.body&&(e(document.body).append(p),p.value="\n\n\n",p.scrollTop=9e4,t=p.scrollHeight===p.scrollTop+p.clientHeight),this.each(function(){function n(){o=b,p.className=a.className,e.each(r,function(e,t){p.style[t]=f.css(t)})}function l(){var e,s,l;if(o!==b&&n(),!d){d=!0,p.value=b.value+a.append,p.style.overflowY=b.style.overflowY,l=parseInt(b.style.height,10),p.style.width=Math.max(f.width(),0)+"px",t?e=p.scrollHeight:(p.scrollTop=0,p.scrollTop=9e4,e=p.scrollTop);var r=parseInt(f.css("maxHeight"),10);r=r&&r>0?r:9e4,e>r?(e=r,s="scroll"):u>e&&(e=u),e+=x,b.style.overflowY=s||i,l!==e&&(b.style.height=e+"px",w&&a.callback.call(b)),setTimeout(function(){d=!1},1)}}var u,d,g,b=this,f=e(b),x=0,w=e.isFunction(a.callback);f.data("autosize")||((f.css("box-sizing")===s||f.css("-moz-box-sizing")===s||f.css("-webkit-box-sizing")===s)&&(x=f.outerHeight()-f.height()),u=Math.max(parseInt(f.css("minHeight"),10)-x,f.height()),g="none"===f.css("resize")||"vertical"===f.css("resize")?"none":"horizontal",f.css({overflow:i,overflowY:i,wordWrap:"break-word",resize:g}).data("autosize",!0),h in b?c in b?b[c]=b.onkeyup=l:b[h]=l:b[c]=l,e(window).on("resize",function(){d=!1,l()}),f.on("autosize",function(){d=!1,l()}),l())})}})(window.jQuery||window.Zepto);
diff --git a/asterix-app/src/main/resources/webui/static/js/jquery.min.js b/asterix-app/src/main/resources/webui/static/js/jquery.min.js
index 006e953..0292c87 100644
--- a/asterix-app/src/main/resources/webui/static/js/jquery.min.js
+++ b/asterix-app/src/main/resources/webui/static/js/jquery.min.js
@@ -1,5 +1,3 @@
-/*! jQuery v1.9.1 | (c) 2005, 2012 jQuery Foundation, Inc. | jquery.org/license
-//@ sourceMappingURL=jquery.min.map
-*/(function(e,t){var n,r,i=typeof t,o=e.document,a=e.location,s=e.jQuery,u=e.$,l={},c=[],p="1.9.1",f=c.concat,d=c.push,h=c.slice,g=c.indexOf,m=l.toString,y=l.hasOwnProperty,v=p.trim,b=function(e,t){return new b.fn.init(e,t,r)},x=/[+-]?(?:\d*\.|)\d+(?:[eE][+-]?\d+|)/.source,w=/\S+/g,T=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,N=/^(?:(<[\w\W]+>)[^>]*|#([\w-]*))$/,C=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,k=/^[\],:{}\s]*$/,E=/(?:^|:|,)(?:\s*\[)+/g,S=/\\(?:["\\\/bfnrt]|u[\da-fA-F]{4})/g,A=/"[^"\\\r\n]*"|true|false|null|-?(?:\d+\.|)\d+(?:[eE][+-]?\d+|)/g,j=/^-ms-/,D=/-([\da-z])/gi,L=function(e,t){return t.toUpperCase()},H=function(e){(o.addEventListener||"load"===e.type||"complete"===o.readyState)&&(q(),b.ready())},q=function(){o.addEventListener?(o.removeEventListener("DOMContentLoaded",H,!1),e.removeEventListener("load",H,!1)):(o.detachEvent("onreadystatechange",H),e.detachEvent("onload",H))};b.fn=b.prototype={jquery:p,constructor:b,init:function(e,n,r){var i,a;if(!e)return this;if("string"==typeof e){if(i="<"===e.charAt(0)&&">"===e.charAt(e.length-1)&&e.length>=3?[null,e,null]:N.exec(e),!i||!i[1]&&n)return!n||n.jquery?(n||r).find(e):this.constructor(n).find(e);if(i[1]){if(n=n instanceof b?n[0]:n,b.merge(this,b.parseHTML(i[1],n&&n.nodeType?n.ownerDocument||n:o,!0)),C.test(i[1])&&b.isPlainObject(n))for(i in n)b.isFunction(this[i])?this[i](n[i]):this.attr(i,n[i]);return this}if(a=o.getElementById(i[2]),a&&a.parentNode){if(a.id!==i[2])return r.find(e);this.length=1,this[0]=a}return this.context=o,this.selector=e,this}return e.nodeType?(this.context=this[0]=e,this.length=1,this):b.isFunction(e)?r.ready(e):(e.selector!==t&&(this.selector=e.selector,this.context=e.context),b.makeArray(e,this))},selector:"",length:0,size:function(){return this.length},toArray:function(){return h.call(this)},get:function(e){return null==e?this.toArray():0>e?this[this.length+e]:this[e]},pushStack:function(e){var t=b.merge(this.constructor(),e);return t.prevObject=this,t.context=this.context,t},each:function(e,t){return b.each(this,e,t)},ready:function(e){return b.ready.promise().done(e),this},slice:function(){return this.pushStack(h.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(e){var t=this.length,n=+e+(0>e?t:0);return this.pushStack(n>=0&&t>n?[this[n]]:[])},map:function(e){return this.pushStack(b.map(this,function(t,n){return e.call(t,n,t)}))},end:function(){return this.prevObject||this.constructor(null)},push:d,sort:[].sort,splice:[].splice},b.fn.init.prototype=b.fn,b.extend=b.fn.extend=function(){var e,n,r,i,o,a,s=arguments[0]||{},u=1,l=arguments.length,c=!1;for("boolean"==typeof s&&(c=s,s=arguments[1]||{},u=2),"object"==typeof s||b.isFunction(s)||(s={}),l===u&&(s=this,--u);l>u;u++)if(null!=(o=arguments[u]))for(i in o)e=s[i],r=o[i],s!==r&&(c&&r&&(b.isPlainObject(r)||(n=b.isArray(r)))?(n?(n=!1,a=e&&b.isArray(e)?e:[]):a=e&&b.isPlainObject(e)?e:{},s[i]=b.extend(c,a,r)):r!==t&&(s[i]=r));return s},b.extend({noConflict:function(t){return e.$===b&&(e.$=u),t&&e.jQuery===b&&(e.jQuery=s),b},isReady:!1,readyWait:1,holdReady:function(e){e?b.readyWait++:b.ready(!0)},ready:function(e){if(e===!0?!--b.readyWait:!b.isReady){if(!o.body)return setTimeout(b.ready);b.isReady=!0,e!==!0&&--b.readyWait>0||(n.resolveWith(o,[b]),b.fn.trigger&&b(o).trigger("ready").off("ready"))}},isFunction:function(e){return"function"===b.type(e)},isArray:Array.isArray||function(e){return"array"===b.type(e)},isWindow:function(e){return null!=e&&e==e.window},isNumeric:function(e){return!isNaN(parseFloat(e))&&isFinite(e)},type:function(e){return null==e?e+"":"object"==typeof e||"function"==typeof e?l[m.call(e)]||"object":typeof e},isPlainObject:function(e){if(!e||"object"!==b.type(e)||e.nodeType||b.isWindow(e))return!1;try{if(e.constructor&&!y.call(e,"constructor")&&!y.call(e.constructor.prototype,"isPrototypeOf"))return!1}catch(n){return!1}var r;for(r in e);return r===t||y.call(e,r)},isEmptyObject:function(e){var t;for(t in e)return!1;return!0},error:function(e){throw Error(e)},parseHTML:function(e,t,n){if(!e||"string"!=typeof e)return null;"boolean"==typeof t&&(n=t,t=!1),t=t||o;var r=C.exec(e),i=!n&&[];return r?[t.createElement(r[1])]:(r=b.buildFragment([e],t,i),i&&b(i).remove(),b.merge([],r.childNodes))},parseJSON:function(n){return e.JSON&&e.JSON.parse?e.JSON.parse(n):null===n?n:"string"==typeof n&&(n=b.trim(n),n&&k.test(n.replace(S,"@").replace(A,"]").replace(E,"")))?Function("return "+n)():(b.error("Invalid JSON: "+n),t)},parseXML:function(n){var r,i;if(!n||"string"!=typeof n)return null;try{e.DOMParser?(i=new DOMParser,r=i.parseFromString(n,"text/xml")):(r=new ActiveXObject("Microsoft.XMLDOM"),r.async="false",r.loadXML(n))}catch(o){r=t}return r&&r.documentElement&&!r.getElementsByTagName("parsererror").length||b.error("Invalid XML: "+n),r},noop:function(){},globalEval:function(t){t&&b.trim(t)&&(e.execScript||function(t){e.eval.call(e,t)})(t)},camelCase:function(e){return e.replace(j,"ms-").replace(D,L)},nodeName:function(e,t){return e.nodeName&&e.nodeName.toLowerCase()===t.toLowerCase()},each:function(e,t,n){var r,i=0,o=e.length,a=M(e);if(n){if(a){for(;o>i;i++)if(r=t.apply(e[i],n),r===!1)break}else for(i in e)if(r=t.apply(e[i],n),r===!1)break}else if(a){for(;o>i;i++)if(r=t.call(e[i],i,e[i]),r===!1)break}else for(i in e)if(r=t.call(e[i],i,e[i]),r===!1)break;return e},trim:v&&!v.call("\ufeff\u00a0")?function(e){return null==e?"":v.call(e)}:function(e){return null==e?"":(e+"").replace(T,"")},makeArray:function(e,t){var n=t||[];return null!=e&&(M(Object(e))?b.merge(n,"string"==typeof e?[e]:e):d.call(n,e)),n},inArray:function(e,t,n){var r;if(t){if(g)return g.call(t,e,n);for(r=t.length,n=n?0>n?Math.max(0,r+n):n:0;r>n;n++)if(n in t&&t[n]===e)return n}return-1},merge:function(e,n){var r=n.length,i=e.length,o=0;if("number"==typeof r)for(;r>o;o++)e[i++]=n[o];else while(n[o]!==t)e[i++]=n[o++];return e.length=i,e},grep:function(e,t,n){var r,i=[],o=0,a=e.length;for(n=!!n;a>o;o++)r=!!t(e[o],o),n!==r&&i.push(e[o]);return i},map:function(e,t,n){var r,i=0,o=e.length,a=M(e),s=[];if(a)for(;o>i;i++)r=t(e[i],i,n),null!=r&&(s[s.length]=r);else for(i in e)r=t(e[i],i,n),null!=r&&(s[s.length]=r);return f.apply([],s)},guid:1,proxy:function(e,n){var r,i,o;return"string"==typeof n&&(o=e[n],n=e,e=o),b.isFunction(e)?(r=h.call(arguments,2),i=function(){return e.apply(n||this,r.concat(h.call(arguments)))},i.guid=e.guid=e.guid||b.guid++,i):t},access:function(e,n,r,i,o,a,s){var u=0,l=e.length,c=null==r;if("object"===b.type(r)){o=!0;for(u in r)b.access(e,n,u,r[u],!0,a,s)}else if(i!==t&&(o=!0,b.isFunction(i)||(s=!0),c&&(s?(n.call(e,i),n=null):(c=n,n=function(e,t,n){return c.call(b(e),n)})),n))for(;l>u;u++)n(e[u],r,s?i:i.call(e[u],u,n(e[u],r)));return o?e:c?n.call(e):l?n(e[0],r):a},now:function(){return(new Date).getTime()}}),b.ready.promise=function(t){if(!n)if(n=b.Deferred(),"complete"===o.readyState)setTimeout(b.ready);else if(o.addEventListener)o.addEventListener("DOMContentLoaded",H,!1),e.addEventListener("load",H,!1);else{o.attachEvent("onreadystatechange",H),e.attachEvent("onload",H);var r=!1;try{r=null==e.frameElement&&o.documentElement}catch(i){}r&&r.doScroll&&function a(){if(!b.isReady){try{r.doScroll("left")}catch(e){return setTimeout(a,50)}q(),b.ready()}}()}return n.promise(t)},b.each("Boolean Number String Function Array Date RegExp Object Error".split(" "),function(e,t){l["[object "+t+"]"]=t.toLowerCase()});function M(e){var t=e.length,n=b.type(e);return b.isWindow(e)?!1:1===e.nodeType&&t?!0:"array"===n||"function"!==n&&(0===t||"number"==typeof t&&t>0&&t-1 in e)}r=b(o);var _={};function F(e){var t=_[e]={};return b.each(e.match(w)||[],function(e,n){t[n]=!0}),t}b.Callbacks=function(e){e="string"==typeof e?_[e]||F(e):b.extend({},e);var n,r,i,o,a,s,u=[],l=!e.once&&[],c=function(t){for(r=e.memory&&t,i=!0,a=s||0,s=0,o=u.length,n=!0;u&&o>a;a++)if(u[a].apply(t[0],t[1])===!1&&e.stopOnFalse){r=!1;break}n=!1,u&&(l?l.length&&c(l.shift()):r?u=[]:p.disable())},p={add:function(){if(u){var t=u.length;(function i(t){b.each(t,function(t,n){var r=b.type(n);"function"===r?e.unique&&p.has(n)||u.push(n):n&&n.length&&"string"!==r&&i(n)})})(arguments),n?o=u.length:r&&(s=t,c(r))}return this},remove:function(){return u&&b.each(arguments,function(e,t){var r;while((r=b.inArray(t,u,r))>-1)u.splice(r,1),n&&(o>=r&&o--,a>=r&&a--)}),this},has:function(e){return e?b.inArray(e,u)>-1:!(!u||!u.length)},empty:function(){return u=[],this},disable:function(){return u=l=r=t,this},disabled:function(){return!u},lock:function(){return l=t,r||p.disable(),this},locked:function(){return!l},fireWith:function(e,t){return t=t||[],t=[e,t.slice?t.slice():t],!u||i&&!l||(n?l.push(t):c(t)),this},fire:function(){return p.fireWith(this,arguments),this},fired:function(){return!!i}};return p},b.extend({Deferred:function(e){var t=[["resolve","done",b.Callbacks("once memory"),"resolved"],["reject","fail",b.Callbacks("once memory"),"rejected"],["notify","progress",b.Callbacks("memory")]],n="pending",r={state:function(){return n},always:function(){return i.done(arguments).fail(arguments),this},then:function(){var e=arguments;return b.Deferred(function(n){b.each(t,function(t,o){var a=o[0],s=b.isFunction(e[t])&&e[t];i[o[1]](function(){var e=s&&s.apply(this,arguments);e&&b.isFunction(e.promise)?e.promise().done(n.resolve).fail(n.reject).progress(n.notify):n[a+"With"](this===r?n.promise():this,s?[e]:arguments)})}),e=null}).promise()},promise:function(e){return null!=e?b.extend(e,r):r}},i={};return r.pipe=r.then,b.each(t,function(e,o){var a=o[2],s=o[3];r[o[1]]=a.add,s&&a.add(function(){n=s},t[1^e][2].disable,t[2][2].lock),i[o[0]]=function(){return i[o[0]+"With"](this===i?r:this,arguments),this},i[o[0]+"With"]=a.fireWith}),r.promise(i),e&&e.call(i,i),i},when:function(e){var t=0,n=h.call(arguments),r=n.length,i=1!==r||e&&b.isFunction(e.promise)?r:0,o=1===i?e:b.Deferred(),a=function(e,t,n){return function(r){t[e]=this,n[e]=arguments.length>1?h.call(arguments):r,n===s?o.notifyWith(t,n):--i||o.resolveWith(t,n)}},s,u,l;if(r>1)for(s=Array(r),u=Array(r),l=Array(r);r>t;t++)n[t]&&b.isFunction(n[t].promise)?n[t].promise().done(a(t,l,n)).fail(o.reject).progress(a(t,u,s)):--i;return i||o.resolveWith(l,n),o.promise()}}),b.support=function(){var t,n,r,a,s,u,l,c,p,f,d=o.createElement("div");if(d.setAttribute("className","t"),d.innerHTML=" <link/><table></table><a href='/a'>a</a><input type='checkbox'/>",n=d.getElementsByTagName("*"),r=d.getElementsByTagName("a")[0],!n||!r||!n.length)return{};s=o.createElement("select"),l=s.appendChild(o.createElement("option")),a=d.getElementsByTagName("input")[0],r.style.cssText="top:1px;float:left;opacity:.5",t={getSetAttribute:"t"!==d.className,leadingWhitespace:3===d.firstChild.nodeType,tbody:!d.getElementsByTagName("tbody").length,htmlSerialize:!!d.getElementsByTagName("link").length,style:/top/.test(r.getAttribute("style")),hrefNormalized:"/a"===r.getAttribute("href"),opacity:/^0.5/.test(r.style.opacity),cssFloat:!!r.style.cssFloat,checkOn:!!a.value,optSelected:l.selected,enctype:!!o.createElement("form").enctype,html5Clone:"<:nav></:nav>"!==o.createElement("nav").cloneNode(!0).outerHTML,boxModel:"CSS1Compat"===o.compatMode,deleteExpando:!0,noCloneEvent:!0,inlineBlockNeedsLayout:!1,shrinkWrapBlocks:!1,reliableMarginRight:!0,boxSizingReliable:!0,pixelPosition:!1},a.checked=!0,t.noCloneChecked=a.cloneNode(!0).checked,s.disabled=!0,t.optDisabled=!l.disabled;try{delete d.test}catch(h){t.deleteExpando=!1}a=o.createElement("input"),a.setAttribute("value",""),t.input=""===a.getAttribute("value"),a.value="t",a.setAttribute("type","radio"),t.radioValue="t"===a.value,a.setAttribute("checked","t"),a.setAttribute("name","t"),u=o.createDocumentFragment(),u.appendChild(a),t.appendChecked=a.checked,t.checkClone=u.cloneNode(!0).cloneNode(!0).lastChild.checked,d.attachEvent&&(d.attachEvent("onclick",function(){t.noCloneEvent=!1}),d.cloneNode(!0).click());for(f in{submit:!0,change:!0,focusin:!0})d.setAttribute(c="on"+f,"t"),t[f+"Bubbles"]=c in e||d.attributes[c].expando===!1;return d.style.backgroundClip="content-box",d.cloneNode(!0).style.backgroundClip="",t.clearCloneStyle="content-box"===d.style.backgroundClip,b(function(){var n,r,a,s="padding:0;margin:0;border:0;display:block;box-sizing:content-box;-moz-box-sizing:content-box;-webkit-box-sizing:content-box;",u=o.getElementsByTagName("body")[0];u&&(n=o.createElement("div"),n.style.cssText="border:0;width:0;height:0;position:absolute;top:0;left:-9999px;margin-top:1px",u.appendChild(n).appendChild(d),d.innerHTML="<table><tr><td></td><td>t</td></tr></table>",a=d.getElementsByTagName("td"),a[0].style.cssText="padding:0;margin:0;border:0;display:none",p=0===a[0].offsetHeight,a[0].style.display="",a[1].style.display="none",t.reliableHiddenOffsets=p&&0===a[0].offsetHeight,d.innerHTML="",d.style.cssText="box-sizing:border-box;-moz-box-sizing:border-box;-webkit-box-sizing:border-box;padding:1px;border:1px;display:block;width:4px;margin-top:1%;position:absolute;top:1%;",t.boxSizing=4===d.offsetWidth,t.doesNotIncludeMarginInBodyOffset=1!==u.offsetTop,e.getComputedStyle&&(t.pixelPosition="1%"!==(e.getComputedStyle(d,null)||{}).top,t.boxSizingReliable="4px"===(e.getComputedStyle(d,null)||{width:"4px"}).width,r=d.appendChild(o.createElement("div")),r.style.cssText=d.style.cssText=s,r.style.marginRight=r.style.width="0",d.style.width="1px",t.reliableMarginRight=!parseFloat((e.getComputedStyle(r,null)||{}).marginRight)),typeof d.style.zoom!==i&&(d.innerHTML="",d.style.cssText=s+"width:1px;padding:1px;display:inline;zoom:1",t.inlineBlockNeedsLayout=3===d.offsetWidth,d.style.display="block",d.innerHTML="<div></div>",d.firstChild.style.width="5px",t.shrinkWrapBlocks=3!==d.offsetWidth,t.inlineBlockNeedsLayout&&(u.style.zoom=1)),u.removeChild(n),n=d=a=r=null)}),n=s=u=l=r=a=null,t}();var O=/(?:\{[\s\S]*\}|\[[\s\S]*\])$/,B=/([A-Z])/g;function P(e,n,r,i){if(b.acceptData(e)){var o,a,s=b.expando,u="string"==typeof n,l=e.nodeType,p=l?b.cache:e,f=l?e[s]:e[s]&&s;if(f&&p[f]&&(i||p[f].data)||!u||r!==t)return f||(l?e[s]=f=c.pop()||b.guid++:f=s),p[f]||(p[f]={},l||(p[f].toJSON=b.noop)),("object"==typeof n||"function"==typeof n)&&(i?p[f]=b.extend(p[f],n):p[f].data=b.extend(p[f].data,n)),o=p[f],i||(o.data||(o.data={}),o=o.data),r!==t&&(o[b.camelCase(n)]=r),u?(a=o[n],null==a&&(a=o[b.camelCase(n)])):a=o,a}}function R(e,t,n){if(b.acceptData(e)){var r,i,o,a=e.nodeType,s=a?b.cache:e,u=a?e[b.expando]:b.expando;if(s[u]){if(t&&(o=n?s[u]:s[u].data)){b.isArray(t)?t=t.concat(b.map(t,b.camelCase)):t in o?t=[t]:(t=b.camelCase(t),t=t in o?[t]:t.split(" "));for(r=0,i=t.length;i>r;r++)delete o[t[r]];if(!(n?$:b.isEmptyObject)(o))return}(n||(delete s[u].data,$(s[u])))&&(a?b.cleanData([e],!0):b.support.deleteExpando||s!=s.window?delete s[u]:s[u]=null)}}}b.extend({cache:{},expando:"jQuery"+(p+Math.random()).replace(/\D/g,""),noData:{embed:!0,object:"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000",applet:!0},hasData:function(e){return e=e.nodeType?b.cache[e[b.expando]]:e[b.expando],!!e&&!$(e)},data:function(e,t,n){return P(e,t,n)},removeData:function(e,t){return R(e,t)},_data:function(e,t,n){return P(e,t,n,!0)},_removeData:function(e,t){return R(e,t,!0)},acceptData:function(e){if(e.nodeType&&1!==e.nodeType&&9!==e.nodeType)return!1;var t=e.nodeName&&b.noData[e.nodeName.toLowerCase()];return!t||t!==!0&&e.getAttribute("classid")===t}}),b.fn.extend({data:function(e,n){var r,i,o=this[0],a=0,s=null;if(e===t){if(this.length&&(s=b.data(o),1===o.nodeType&&!b._data(o,"parsedAttrs"))){for(r=o.attributes;r.length>a;a++)i=r[a].name,i.indexOf("data-")||(i=b.camelCase(i.slice(5)),W(o,i,s[i]));b._data(o,"parsedAttrs",!0)}return s}return"object"==typeof e?this.each(function(){b.data(this,e)}):b.access(this,function(n){return n===t?o?W(o,e,b.data(o,e)):null:(this.each(function(){b.data(this,e,n)}),t)},null,n,arguments.length>1,null,!0)},removeData:function(e){return this.each(function(){b.removeData(this,e)})}});function W(e,n,r){if(r===t&&1===e.nodeType){var i="data-"+n.replace(B,"-$1").toLowerCase();if(r=e.getAttribute(i),"string"==typeof r){try{r="true"===r?!0:"false"===r?!1:"null"===r?null:+r+""===r?+r:O.test(r)?b.parseJSON(r):r}catch(o){}b.data(e,n,r)}else r=t}return r}function $(e){var t;for(t in e)if(("data"!==t||!b.isEmptyObject(e[t]))&&"toJSON"!==t)return!1;return!0}b.extend({queue:function(e,n,r){var i;return e?(n=(n||"fx")+"queue",i=b._data(e,n),r&&(!i||b.isArray(r)?i=b._data(e,n,b.makeArray(r)):i.push(r)),i||[]):t},dequeue:function(e,t){t=t||"fx";var n=b.queue(e,t),r=n.length,i=n.shift(),o=b._queueHooks(e,t),a=function(){b.dequeue(e,t)};"inprogress"===i&&(i=n.shift(),r--),o.cur=i,i&&("fx"===t&&n.unshift("inprogress"),delete o.stop,i.call(e,a,o)),!r&&o&&o.empty.fire()},_queueHooks:function(e,t){var n=t+"queueHooks";return b._data(e,n)||b._data(e,n,{empty:b.Callbacks("once memory").add(function(){b._removeData(e,t+"queue"),b._removeData(e,n)})})}}),b.fn.extend({queue:function(e,n){var r=2;return"string"!=typeof e&&(n=e,e="fx",r--),r>arguments.length?b.queue(this[0],e):n===t?this:this.each(function(){var t=b.queue(this,e,n);b._queueHooks(this,e),"fx"===e&&"inprogress"!==t[0]&&b.dequeue(this,e)})},dequeue:function(e){return this.each(function(){b.dequeue(this,e)})},delay:function(e,t){return e=b.fx?b.fx.speeds[e]||e:e,t=t||"fx",this.queue(t,function(t,n){var r=setTimeout(t,e);n.stop=function(){clearTimeout(r)}})},clearQueue:function(e){return this.queue(e||"fx",[])},promise:function(e,n){var r,i=1,o=b.Deferred(),a=this,s=this.length,u=function(){--i||o.resolveWith(a,[a])};"string"!=typeof e&&(n=e,e=t),e=e||"fx";while(s--)r=b._data(a[s],e+"queueHooks"),r&&r.empty&&(i++,r.empty.add(u));return u(),o.promise(n)}});var I,z,X=/[\t\r\n]/g,U=/\r/g,V=/^(?:input|select|textarea|button|object)$/i,Y=/^(?:a|area)$/i,J=/^(?:checked|selected|autofocus|autoplay|async|controls|defer|disabled|hidden|loop|multiple|open|readonly|required|scoped)$/i,G=/^(?:checked|selected)$/i,Q=b.support.getSetAttribute,K=b.support.input;b.fn.extend({attr:function(e,t){return b.access(this,b.attr,e,t,arguments.length>1)},removeAttr:function(e){return this.each(function(){b.removeAttr(this,e)})},prop:function(e,t){return b.access(this,b.prop,e,t,arguments.length>1)},removeProp:function(e){return e=b.propFix[e]||e,this.each(function(){try{this[e]=t,delete this[e]}catch(n){}})},addClass:function(e){var t,n,r,i,o,a=0,s=this.length,u="string"==typeof e&&e;if(b.isFunction(e))return this.each(function(t){b(this).addClass(e.call(this,t,this.className))});if(u)for(t=(e||"").match(w)||[];s>a;a++)if(n=this[a],r=1===n.nodeType&&(n.className?(" "+n.className+" ").replace(X," "):" ")){o=0;while(i=t[o++])0>r.indexOf(" "+i+" ")&&(r+=i+" ");n.className=b.trim(r)}return this},removeClass:function(e){var t,n,r,i,o,a=0,s=this.length,u=0===arguments.length||"string"==typeof e&&e;if(b.isFunction(e))return this.each(function(t){b(this).removeClass(e.call(this,t,this.className))});if(u)for(t=(e||"").match(w)||[];s>a;a++)if(n=this[a],r=1===n.nodeType&&(n.className?(" "+n.className+" ").replace(X," "):"")){o=0;while(i=t[o++])while(r.indexOf(" "+i+" ")>=0)r=r.replace(" "+i+" "," ");n.className=e?b.trim(r):""}return this},toggleClass:function(e,t){var n=typeof e,r="boolean"==typeof t;return b.isFunction(e)?this.each(function(n){b(this).toggleClass(e.call(this,n,this.className,t),t)}):this.each(function(){if("string"===n){var o,a=0,s=b(this),u=t,l=e.match(w)||[];while(o=l[a++])u=r?u:!s.hasClass(o),s[u?"addClass":"removeClass"](o)}else(n===i||"boolean"===n)&&(this.className&&b._data(this,"__className__",this.className),this.className=this.className||e===!1?"":b._data(this,"__className__")||"")})},hasClass:function(e){var t=" "+e+" ",n=0,r=this.length;for(;r>n;n++)if(1===this[n].nodeType&&(" "+this[n].className+" ").replace(X," ").indexOf(t)>=0)return!0;return!1},val:function(e){var n,r,i,o=this[0];{if(arguments.length)return i=b.isFunction(e),this.each(function(n){var o,a=b(this);1===this.nodeType&&(o=i?e.call(this,n,a.val()):e,null==o?o="":"number"==typeof o?o+="":b.isArray(o)&&(o=b.map(o,function(e){return null==e?"":e+""})),r=b.valHooks[this.type]||b.valHooks[this.nodeName.toLowerCase()],r&&"set"in r&&r.set(this,o,"value")!==t||(this.value=o))});if(o)return r=b.valHooks[o.type]||b.valHooks[o.nodeName.toLowerCase()],r&&"get"in r&&(n=r.get(o,"value"))!==t?n:(n=o.value,"string"==typeof n?n.replace(U,""):null==n?"":n)}}}),b.extend({valHooks:{option:{get:function(e){var t=e.attributes.value;return!t||t.specified?e.value:e.text}},select:{get:function(e){var t,n,r=e.options,i=e.selectedIndex,o="select-one"===e.type||0>i,a=o?null:[],s=o?i+1:r.length,u=0>i?s:o?i:0;for(;s>u;u++)if(n=r[u],!(!n.selected&&u!==i||(b.support.optDisabled?n.disabled:null!==n.getAttribute("disabled"))||n.parentNode.disabled&&b.nodeName(n.parentNode,"optgroup"))){if(t=b(n).val(),o)return t;a.push(t)}return a},set:function(e,t){var n=b.makeArray(t);return b(e).find("option").each(function(){this.selected=b.inArray(b(this).val(),n)>=0}),n.length||(e.selectedIndex=-1),n}}},attr:function(e,n,r){var o,a,s,u=e.nodeType;if(e&&3!==u&&8!==u&&2!==u)return typeof e.getAttribute===i?b.prop(e,n,r):(a=1!==u||!b.isXMLDoc(e),a&&(n=n.toLowerCase(),o=b.attrHooks[n]||(J.test(n)?z:I)),r===t?o&&a&&"get"in o&&null!==(s=o.get(e,n))?s:(typeof e.getAttribute!==i&&(s=e.getAttribute(n)),null==s?t:s):null!==r?o&&a&&"set"in o&&(s=o.set(e,r,n))!==t?s:(e.setAttribute(n,r+""),r):(b.removeAttr(e,n),t))},removeAttr:function(e,t){var n,r,i=0,o=t&&t.match(w);if(o&&1===e.nodeType)while(n=o[i++])r=b.propFix[n]||n,J.test(n)?!Q&&G.test(n)?e[b.camelCase("default-"+n)]=e[r]=!1:e[r]=!1:b.attr(e,n,""),e.removeAttribute(Q?n:r)},attrHooks:{type:{set:function(e,t){if(!b.support.radioValue&&"radio"===t&&b.nodeName(e,"input")){var n=e.value;return e.setAttribute("type",t),n&&(e.value=n),t}}}},propFix:{tabindex:"tabIndex",readonly:"readOnly","for":"htmlFor","class":"className",maxlength:"maxLength",cellspacing:"cellSpacing",cellpadding:"cellPadding",rowspan:"rowSpan",colspan:"colSpan",usemap:"useMap",frameborder:"frameBorder",contenteditable:"contentEditable"},prop:function(e,n,r){var i,o,a,s=e.nodeType;if(e&&3!==s&&8!==s&&2!==s)return a=1!==s||!b.isXMLDoc(e),a&&(n=b.propFix[n]||n,o=b.propHooks[n]),r!==t?o&&"set"in o&&(i=o.set(e,r,n))!==t?i:e[n]=r:o&&"get"in o&&null!==(i=o.get(e,n))?i:e[n]},propHooks:{tabIndex:{get:function(e){var n=e.getAttributeNode("tabindex");return n&&n.specified?parseInt(n.value,10):V.test(e.nodeName)||Y.test(e.nodeName)&&e.href?0:t}}}}),z={get:function(e,n){var r=b.prop(e,n),i="boolean"==typeof r&&e.getAttribute(n),o="boolean"==typeof r?K&&Q?null!=i:G.test(n)?e[b.camelCase("default-"+n)]:!!i:e.getAttributeNode(n);return o&&o.value!==!1?n.toLowerCase():t},set:function(e,t,n){return t===!1?b.removeAttr(e,n):K&&Q||!G.test(n)?e.setAttribute(!Q&&b.propFix[n]||n,n):e[b.camelCase("default-"+n)]=e[n]=!0,n}},K&&Q||(b.attrHooks.value={get:function(e,n){var r=e.getAttributeNode(n);return b.nodeName(e,"input")?e.defaultValue:r&&r.specified?r.value:t},set:function(e,n,r){return b.nodeName(e,"input")?(e.defaultValue=n,t):I&&I.set(e,n,r)}}),Q||(I=b.valHooks.button={get:function(e,n){var r=e.getAttributeNode(n);return r&&("id"===n||"name"===n||"coords"===n?""!==r.value:r.specified)?r.value:t},set:function(e,n,r){var i=e.getAttributeNode(r);return i||e.setAttributeNode(i=e.ownerDocument.createAttribute(r)),i.value=n+="","value"===r||n===e.getAttribute(r)?n:t}},b.attrHooks.contenteditable={get:I.get,set:function(e,t,n){I.set(e,""===t?!1:t,n)}},b.each(["width","height"],function(e,n){b.attrHooks[n]=b.extend(b.attrHooks[n],{set:function(e,r){return""===r?(e.setAttribute(n,"auto"),r):t}})})),b.support.hrefNormalized||(b.each(["href","src","width","height"],function(e,n){b.attrHooks[n]=b.extend(b.attrHooks[n],{get:function(e){var r=e.getAttribute(n,2);return null==r?t:r}})}),b.each(["href","src"],function(e,t){b.propHooks[t]={get:function(e){return e.getAttribute(t,4)}}})),b.support.style||(b.attrHooks.style={get:function(e){return e.style.cssText||t},set:function(e,t){return e.style.cssText=t+""}}),b.support.optSelected||(b.propHooks.selected=b.extend(b.propHooks.selected,{get:function(e){var t=e.parentNode;return t&&(t.selectedIndex,t.parentNode&&t.parentNode.selectedIndex),null}})),b.support.enctype||(b.propFix.enctype="encoding"),b.support.checkOn||b.each(["radio","checkbox"],function(){b.valHooks[this]={get:function(e){return null===e.getAttribute("value")?"on":e.value}}}),b.each(["radio","checkbox"],function(){b.valHooks[this]=b.extend(b.valHooks[this],{set:function(e,n){return b.isArray(n)?e.checked=b.inArray(b(e).val(),n)>=0:t}})});var Z=/^(?:input|select|textarea)$/i,et=/^key/,tt=/^(?:mouse|contextmenu)|click/,nt=/^(?:focusinfocus|focusoutblur)$/,rt=/^([^.]*)(?:\.(.+)|)$/;function it(){return!0}function ot(){return!1}b.event={global:{},add:function(e,n,r,o,a){var s,u,l,c,p,f,d,h,g,m,y,v=b._data(e);if(v){r.handler&&(c=r,r=c.handler,a=c.selector),r.guid||(r.guid=b.guid++),(u=v.events)||(u=v.events={}),(f=v.handle)||(f=v.handle=function(e){return typeof b===i||e&&b.event.triggered===e.type?t:b.event.dispatch.apply(f.elem,arguments)},f.elem=e),n=(n||"").match(w)||[""],l=n.length;while(l--)s=rt.exec(n[l])||[],g=y=s[1],m=(s[2]||"").split(".").sort(),p=b.event.special[g]||{},g=(a?p.delegateType:p.bindType)||g,p=b.event.special[g]||{},d=b.extend({type:g,origType:y,data:o,handler:r,guid:r.guid,selector:a,needsContext:a&&b.expr.match.needsContext.test(a),namespace:m.join(".")},c),(h=u[g])||(h=u[g]=[],h.delegateCount=0,p.setup&&p.setup.call(e,o,m,f)!==!1||(e.addEventListener?e.addEventListener(g,f,!1):e.attachEvent&&e.attachEvent("on"+g,f))),p.add&&(p.add.call(e,d),d.handler.guid||(d.handler.guid=r.guid)),a?h.splice(h.delegateCount++,0,d):h.push(d),b.event.global[g]=!0;e=null}},remove:function(e,t,n,r,i){var o,a,s,u,l,c,p,f,d,h,g,m=b.hasData(e)&&b._data(e);if(m&&(c=m.events)){t=(t||"").match(w)||[""],l=t.length;while(l--)if(s=rt.exec(t[l])||[],d=g=s[1],h=(s[2]||"").split(".").sort(),d){p=b.event.special[d]||{},d=(r?p.delegateType:p.bindType)||d,f=c[d]||[],s=s[2]&&RegExp("(^|\\.)"+h.join("\\.(?:.*\\.|)")+"(\\.|$)"),u=o=f.length;while(o--)a=f[o],!i&&g!==a.origType||n&&n.guid!==a.guid||s&&!s.test(a.namespace)||r&&r!==a.selector&&("**"!==r||!a.selector)||(f.splice(o,1),a.selector&&f.delegateCount--,p.remove&&p.remove.call(e,a));u&&!f.length&&(p.teardown&&p.teardown.call(e,h,m.handle)!==!1||b.removeEvent(e,d,m.handle),delete c[d])}else for(d in c)b.event.remove(e,d+t[l],n,r,!0);b.isEmptyObject(c)&&(delete m.handle,b._removeData(e,"events"))}},trigger:function(n,r,i,a){var s,u,l,c,p,f,d,h=[i||o],g=y.call(n,"type")?n.type:n,m=y.call(n,"namespace")?n.namespace.split("."):[];if(l=f=i=i||o,3!==i.nodeType&&8!==i.nodeType&&!nt.test(g+b.event.triggered)&&(g.indexOf(".")>=0&&(m=g.split("."),g=m.shift(),m.sort()),u=0>g.indexOf(":")&&"on"+g,n=n[b.expando]?n:new b.Event(g,"object"==typeof n&&n),n.isTrigger=!0,n.namespace=m.join("."),n.namespace_re=n.namespace?RegExp("(^|\\.)"+m.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,n.result=t,n.target||(n.target=i),r=null==r?[n]:b.makeArray(r,[n]),p=b.event.special[g]||{},a||!p.trigger||p.trigger.apply(i,r)!==!1)){if(!a&&!p.noBubble&&!b.isWindow(i)){for(c=p.delegateType||g,nt.test(c+g)||(l=l.parentNode);l;l=l.parentNode)h.push(l),f=l;f===(i.ownerDocument||o)&&h.push(f.defaultView||f.parentWindow||e)}d=0;while((l=h[d++])&&!n.isPropagationStopped())n.type=d>1?c:p.bindType||g,s=(b._data(l,"events")||{})[n.type]&&b._data(l,"handle"),s&&s.apply(l,r),s=u&&l[u],s&&b.acceptData(l)&&s.apply&&s.apply(l,r)===!1&&n.preventDefault();if(n.type=g,!(a||n.isDefaultPrevented()||p._default&&p._default.apply(i.ownerDocument,r)!==!1||"click"===g&&b.nodeName(i,"a")||!b.acceptData(i)||!u||!i[g]||b.isWindow(i))){f=i[u],f&&(i[u]=null),b.event.triggered=g;try{i[g]()}catch(v){}b.event.triggered=t,f&&(i[u]=f)}return n.result}},dispatch:function(e){e=b.event.fix(e);var n,r,i,o,a,s=[],u=h.call(arguments),l=(b._data(this,"events")||{})[e.type]||[],c=b.event.special[e.type]||{};if(u[0]=e,e.delegateTarget=this,!c.preDispatch||c.preDispatch.call(this,e)!==!1){s=b.event.handlers.call(this,e,l),n=0;while((o=s[n++])&&!e.isPropagationStopped()){e.currentTarget=o.elem,a=0;while((i=o.handlers[a++])&&!e.isImmediatePropagationStopped())(!e.namespace_re||e.namespace_re.test(i.namespace))&&(e.handleObj=i,e.data=i.data,r=((b.event.special[i.origType]||{}).handle||i.handler).apply(o.elem,u),r!==t&&(e.result=r)===!1&&(e.preventDefault(),e.stopPropagation()))}return c.postDispatch&&c.postDispatch.call(this,e),e.result}},handlers:function(e,n){var r,i,o,a,s=[],u=n.delegateCount,l=e.target;if(u&&l.nodeType&&(!e.button||"click"!==e.type))for(;l!=this;l=l.parentNode||this)if(1===l.nodeType&&(l.disabled!==!0||"click"!==e.type)){for(o=[],a=0;u>a;a++)i=n[a],r=i.selector+" ",o[r]===t&&(o[r]=i.needsContext?b(r,this).index(l)>=0:b.find(r,this,null,[l]).length),o[r]&&o.push(i);o.length&&s.push({elem:l,handlers:o})}return n.length>u&&s.push({elem:this,handlers:n.slice(u)}),s},fix:function(e){if(e[b.expando])return e;var t,n,r,i=e.type,a=e,s=this.fixHooks[i];s||(this.fixHooks[i]=s=tt.test(i)?this.mouseHooks:et.test(i)?this.keyHooks:{}),r=s.props?this.props.concat(s.props):this.props,e=new b.Event(a),t=r.length;while(t--)n=r[t],e[n]=a[n];return e.target||(e.target=a.srcElement||o),3===e.target.nodeType&&(e.target=e.target.parentNode),e.metaKey=!!e.metaKey,s.filter?s.filter(e,a):e},props:"altKey bubbles cancelable ctrlKey currentTarget eventPhase metaKey relatedTarget shiftKey target timeStamp view which".split(" "),fixHooks:{},keyHooks:{props:"char charCode key keyCode".split(" "),filter:function(e,t){return null==e.which&&(e.which=null!=t.charCode?t.charCode:t.keyCode),e}},mouseHooks:{props:"button buttons clientX clientY fromElement offsetX offsetY pageX pageY screenX screenY toElement".split(" "),filter:function(e,n){var r,i,a,s=n.button,u=n.fromElement;return null==e.pageX&&null!=n.clientX&&(i=e.target.ownerDocument||o,a=i.documentElement,r=i.body,e.pageX=n.clientX+(a&&a.scrollLeft||r&&r.scrollLeft||0)-(a&&a.clientLeft||r&&r.clientLeft||0),e.pageY=n.clientY+(a&&a.scrollTop||r&&r.scrollTop||0)-(a&&a.clientTop||r&&r.clientTop||0)),!e.relatedTarget&&u&&(e.relatedTarget=u===e.target?n.toElement:u),e.which||s===t||(e.which=1&s?1:2&s?3:4&s?2:0),e}},special:{load:{noBubble:!0},click:{trigger:function(){return b.nodeName(this,"input")&&"checkbox"===this.type&&this.click?(this.click(),!1):t}},focus:{trigger:function(){if(this!==o.activeElement&&this.focus)try{return this.focus(),!1}catch(e){}},delegateType:"focusin"},blur:{trigger:function(){return this===o.activeElement&&this.blur?(this.blur(),!1):t},delegateType:"focusout"},beforeunload:{postDispatch:function(e){e.result!==t&&(e.originalEvent.returnValue=e.result)}}},simulate:function(e,t,n,r){var i=b.extend(new b.Event,n,{type:e,isSimulated:!0,originalEvent:{}});r?b.event.trigger(i,null,t):b.event.dispatch.call(t,i),i.isDefaultPrevented()&&n.preventDefault()}},b.removeEvent=o.removeEventListener?function(e,t,n){e.removeEventListener&&e.removeEventListener(t,n,!1)}:function(e,t,n){var r="on"+t;e.detachEvent&&(typeof e[r]===i&&(e[r]=null),e.detachEvent(r,n))},b.Event=function(e,n){return this instanceof b.Event?(e&&e.type?(this.originalEvent=e,this.type=e.type,this.isDefaultPrevented=e.defaultPrevented||e.returnValue===!1||e.getPreventDefault&&e.getPreventDefault()?it:ot):this.type=e,n&&b.extend(this,n),this.timeStamp=e&&e.timeStamp||b.now(),this[b.expando]=!0,t):new b.Event(e,n)},b.Event.prototype={isDefaultPrevented:ot,isPropagationStopped:ot,isImmediatePropagationStopped:ot,preventDefault:function(){var e=this.originalEvent;this.isDefaultPrevented=it,e&&(e.preventDefault?e.preventDefault():e.returnValue=!1)},stopPropagation:function(){var e=this.originalEvent;this.isPropagationStopped=it,e&&(e.stopPropagation&&e.stopPropagation(),e.cancelBubble=!0)},stopImmediatePropagation:function(){this.isImmediatePropagationStopped=it,this.stopPropagation()}},b.each({mouseenter:"mouseover",mouseleave:"mouseout"},function(e,t){b.event.special[e]={delegateType:t,bindType:t,handle:function(e){var n,r=this,i=e.relatedTarget,o=e.handleObj;
+(function(e,t){var n,r,i=typeof t,o=e.document,a=e.location,s=e.jQuery,u=e.$,l={},c=[],p="1.9.1",f=c.concat,d=c.push,h=c.slice,g=c.indexOf,m=l.toString,y=l.hasOwnProperty,v=p.trim,b=function(e,t){return new b.fn.init(e,t,r)},x=/[+-]?(?:\d*\.|)\d+(?:[eE][+-]?\d+|)/.source,w=/\S+/g,T=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,N=/^(?:(<[\w\W]+>)[^>]*|#([\w-]*))$/,C=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,k=/^[\],:{}\s]*$/,E=/(?:^|:|,)(?:\s*\[)+/g,S=/\\(?:["\\\/bfnrt]|u[\da-fA-F]{4})/g,A=/"[^"\\\r\n]*"|true|false|null|-?(?:\d+\.|)\d+(?:[eE][+-]?\d+|)/g,j=/^-ms-/,D=/-([\da-z])/gi,L=function(e,t){return t.toUpperCase()},H=function(e){(o.addEventListener||"load"===e.type||"complete"===o.readyState)&&(q(),b.ready())},q=function(){o.addEventListener?(o.removeEventListener("DOMContentLoaded",H,!1),e.removeEventListener("load",H,!1)):(o.detachEvent("onreadystatechange",H),e.detachEvent("onload",H))};b.fn=b.prototype={jquery:p,constructor:b,init:function(e,n,r){var i,a;if(!e)return this;if("string"==typeof e){if(i="<"===e.charAt(0)&&">"===e.charAt(e.length-1)&&e.length>=3?[null,e,null]:N.exec(e),!i||!i[1]&&n)return!n||n.jquery?(n||r).find(e):this.constructor(n).find(e);if(i[1]){if(n=n instanceof b?n[0]:n,b.merge(this,b.parseHTML(i[1],n&&n.nodeType?n.ownerDocument||n:o,!0)),C.test(i[1])&&b.isPlainObject(n))for(i in n)b.isFunction(this[i])?this[i](n[i]):this.attr(i,n[i]);return this}if(a=o.getElementById(i[2]),a&&a.parentNode){if(a.id!==i[2])return r.find(e);this.length=1,this[0]=a}return this.context=o,this.selector=e,this}return e.nodeType?(this.context=this[0]=e,this.length=1,this):b.isFunction(e)?r.ready(e):(e.selector!==t&&(this.selector=e.selector,this.context=e.context),b.makeArray(e,this))},selector:"",length:0,size:function(){return this.length},toArray:function(){return h.call(this)},get:function(e){return null==e?this.toArray():0>e?this[this.length+e]:this[e]},pushStack:function(e){var t=b.merge(this.constructor(),e);return t.prevObject=this,t.context=this.context,t},each:function(e,t){return b.each(this,e,t)},ready:function(e){return b.ready.promise().done(e),this},slice:function(){return this.pushStack(h.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(e){var t=this.length,n=+e+(0>e?t:0);return this.pushStack(n>=0&&t>n?[this[n]]:[])},map:function(e){return this.pushStack(b.map(this,function(t,n){return e.call(t,n,t)}))},end:function(){return this.prevObject||this.constructor(null)},push:d,sort:[].sort,splice:[].splice},b.fn.init.prototype=b.fn,b.extend=b.fn.extend=function(){var e,n,r,i,o,a,s=arguments[0]||{},u=1,l=arguments.length,c=!1;for("boolean"==typeof s&&(c=s,s=arguments[1]||{},u=2),"object"==typeof s||b.isFunction(s)||(s={}),l===u&&(s=this,--u);l>u;u++)if(null!=(o=arguments[u]))for(i in o)e=s[i],r=o[i],s!==r&&(c&&r&&(b.isPlainObject(r)||(n=b.isArray(r)))?(n?(n=!1,a=e&&b.isArray(e)?e:[]):a=e&&b.isPlainObject(e)?e:{},s[i]=b.extend(c,a,r)):r!==t&&(s[i]=r));return s},b.extend({noConflict:function(t){return e.$===b&&(e.$=u),t&&e.jQuery===b&&(e.jQuery=s),b},isReady:!1,readyWait:1,holdReady:function(e){e?b.readyWait++:b.ready(!0)},ready:function(e){if(e===!0?!--b.readyWait:!b.isReady){if(!o.body)return setTimeout(b.ready);b.isReady=!0,e!==!0&&--b.readyWait>0||(n.resolveWith(o,[b]),b.fn.trigger&&b(o).trigger("ready").off("ready"))}},isFunction:function(e){return"function"===b.type(e)},isArray:Array.isArray||function(e){return"array"===b.type(e)},isWindow:function(e){return null!=e&&e==e.window},isNumeric:function(e){return!isNaN(parseFloat(e))&&isFinite(e)},type:function(e){return null==e?e+"":"object"==typeof e||"function"==typeof e?l[m.call(e)]||"object":typeof e},isPlainObject:function(e){if(!e||"object"!==b.type(e)||e.nodeType||b.isWindow(e))return!1;try{if(e.constructor&&!y.call(e,"constructor")&&!y.call(e.constructor.prototype,"isPrototypeOf"))return!1}catch(n){return!1}var r;for(r in e);return r===t||y.call(e,r)},isEmptyObject:function(e){var t;for(t in e)return!1;return!0},error:function(e){throw Error(e)},parseHTML:function(e,t,n){if(!e||"string"!=typeof e)return null;"boolean"==typeof t&&(n=t,t=!1),t=t||o;var r=C.exec(e),i=!n&&[];return r?[t.createElement(r[1])]:(r=b.buildFragment([e],t,i),i&&b(i).remove(),b.merge([],r.childNodes))},parseJSON:function(n){return e.JSON&&e.JSON.parse?e.JSON.parse(n):null===n?n:"string"==typeof n&&(n=b.trim(n),n&&k.test(n.replace(S,"@").replace(A,"]").replace(E,"")))?Function("return "+n)():(b.error("Invalid JSON: "+n),t)},parseXML:function(n){var r,i;if(!n||"string"!=typeof n)return null;try{e.DOMParser?(i=new DOMParser,r=i.parseFromString(n,"text/xml")):(r=new ActiveXObject("Microsoft.XMLDOM"),r.async="false",r.loadXML(n))}catch(o){r=t}return r&&r.documentElement&&!r.getElementsByTagName("parsererror").length||b.error("Invalid XML: "+n),r},noop:function(){},globalEval:function(t){t&&b.trim(t)&&(e.execScript||function(t){e.eval.call(e,t)})(t)},camelCase:function(e){return e.replace(j,"ms-").replace(D,L)},nodeName:function(e,t){return e.nodeName&&e.nodeName.toLowerCase()===t.toLowerCase()},each:function(e,t,n){var r,i=0,o=e.length,a=M(e);if(n){if(a){for(;o>i;i++)if(r=t.apply(e[i],n),r===!1)break}else for(i in e)if(r=t.apply(e[i],n),r===!1)break}else if(a){for(;o>i;i++)if(r=t.call(e[i],i,e[i]),r===!1)break}else for(i in e)if(r=t.call(e[i],i,e[i]),r===!1)break;return e},trim:v&&!v.call("\ufeff\u00a0")?function(e){return null==e?"":v.call(e)}:function(e){return null==e?"":(e+"").replace(T,"")},makeArray:function(e,t){var n=t||[];return null!=e&&(M(Object(e))?b.merge(n,"string"==typeof e?[e]:e):d.call(n,e)),n},inArray:function(e,t,n){var r;if(t){if(g)return g.call(t,e,n);for(r=t.length,n=n?0>n?Math.max(0,r+n):n:0;r>n;n++)if(n in t&&t[n]===e)return n}return-1},merge:function(e,n){var r=n.length,i=e.length,o=0;if("number"==typeof r)for(;r>o;o++)e[i++]=n[o];else while(n[o]!==t)e[i++]=n[o++];return e.length=i,e},grep:function(e,t,n){var r,i=[],o=0,a=e.length;for(n=!!n;a>o;o++)r=!!t(e[o],o),n!==r&&i.push(e[o]);return i},map:function(e,t,n){var r,i=0,o=e.length,a=M(e),s=[];if(a)for(;o>i;i++)r=t(e[i],i,n),null!=r&&(s[s.length]=r);else for(i in e)r=t(e[i],i,n),null!=r&&(s[s.length]=r);return f.apply([],s)},guid:1,proxy:function(e,n){var r,i,o;return"string"==typeof n&&(o=e[n],n=e,e=o),b.isFunction(e)?(r=h.call(arguments,2),i=function(){return e.apply(n||this,r.concat(h.call(arguments)))},i.guid=e.guid=e.guid||b.guid++,i):t},access:function(e,n,r,i,o,a,s){var u=0,l=e.length,c=null==r;if("object"===b.type(r)){o=!0;for(u in r)b.access(e,n,u,r[u],!0,a,s)}else if(i!==t&&(o=!0,b.isFunction(i)||(s=!0),c&&(s?(n.call(e,i),n=null):(c=n,n=function(e,t,n){return c.call(b(e),n)})),n))for(;l>u;u++)n(e[u],r,s?i:i.call(e[u],u,n(e[u],r)));return o?e:c?n.call(e):l?n(e[0],r):a},now:function(){return(new Date).getTime()}}),b.ready.promise=function(t){if(!n)if(n=b.Deferred(),"complete"===o.readyState)setTimeout(b.ready);else if(o.addEventListener)o.addEventListener("DOMContentLoaded",H,!1),e.addEventListener("load",H,!1);else{o.attachEvent("onreadystatechange",H),e.attachEvent("onload",H);var r=!1;try{r=null==e.frameElement&&o.documentElement}catch(i){}r&&r.doScroll&&function a(){if(!b.isReady){try{r.doScroll("left")}catch(e){return setTimeout(a,50)}q(),b.ready()}}()}return n.promise(t)},b.each("Boolean Number String Function Array Date RegExp Object Error".split(" "),function(e,t){l["[object "+t+"]"]=t.toLowerCase()});function M(e){var t=e.length,n=b.type(e);return b.isWindow(e)?!1:1===e.nodeType&&t?!0:"array"===n||"function"!==n&&(0===t||"number"==typeof t&&t>0&&t-1 in e)}r=b(o);var _={};function F(e){var t=_[e]={};return b.each(e.match(w)||[],function(e,n){t[n]=!0}),t}b.Callbacks=function(e){e="string"==typeof e?_[e]||F(e):b.extend({},e);var n,r,i,o,a,s,u=[],l=!e.once&&[],c=function(t){for(r=e.memory&&t,i=!0,a=s||0,s=0,o=u.length,n=!0;u&&o>a;a++)if(u[a].apply(t[0],t[1])===!1&&e.stopOnFalse){r=!1;break}n=!1,u&&(l?l.length&&c(l.shift()):r?u=[]:p.disable())},p={add:function(){if(u){var t=u.length;(function i(t){b.each(t,function(t,n){var r=b.type(n);"function"===r?e.unique&&p.has(n)||u.push(n):n&&n.length&&"string"!==r&&i(n)})})(arguments),n?o=u.length:r&&(s=t,c(r))}return this},remove:function(){return u&&b.each(arguments,function(e,t){var r;while((r=b.inArray(t,u,r))>-1)u.splice(r,1),n&&(o>=r&&o--,a>=r&&a--)}),this},has:function(e){return e?b.inArray(e,u)>-1:!(!u||!u.length)},empty:function(){return u=[],this},disable:function(){return u=l=r=t,this},disabled:function(){return!u},lock:function(){return l=t,r||p.disable(),this},locked:function(){return!l},fireWith:function(e,t){return t=t||[],t=[e,t.slice?t.slice():t],!u||i&&!l||(n?l.push(t):c(t)),this},fire:function(){return p.fireWith(this,arguments),this},fired:function(){return!!i}};return p},b.extend({Deferred:function(e){var t=[["resolve","done",b.Callbacks("once memory"),"resolved"],["reject","fail",b.Callbacks("once memory"),"rejected"],["notify","progress",b.Callbacks("memory")]],n="pending",r={state:function(){return n},always:function(){return i.done(arguments).fail(arguments),this},then:function(){var e=arguments;return b.Deferred(function(n){b.each(t,function(t,o){var a=o[0],s=b.isFunction(e[t])&&e[t];i[o[1]](function(){var e=s&&s.apply(this,arguments);e&&b.isFunction(e.promise)?e.promise().done(n.resolve).fail(n.reject).progress(n.notify):n[a+"With"](this===r?n.promise():this,s?[e]:arguments)})}),e=null}).promise()},promise:function(e){return null!=e?b.extend(e,r):r}},i={};return r.pipe=r.then,b.each(t,function(e,o){var a=o[2],s=o[3];r[o[1]]=a.add,s&&a.add(function(){n=s},t[1^e][2].disable,t[2][2].lock),i[o[0]]=function(){return i[o[0]+"With"](this===i?r:this,arguments),this},i[o[0]+"With"]=a.fireWith}),r.promise(i),e&&e.call(i,i),i},when:function(e){var t=0,n=h.call(arguments),r=n.length,i=1!==r||e&&b.isFunction(e.promise)?r:0,o=1===i?e:b.Deferred(),a=function(e,t,n){return function(r){t[e]=this,n[e]=arguments.length>1?h.call(arguments):r,n===s?o.notifyWith(t,n):--i||o.resolveWith(t,n)}},s,u,l;if(r>1)for(s=Array(r),u=Array(r),l=Array(r);r>t;t++)n[t]&&b.isFunction(n[t].promise)?n[t].promise().done(a(t,l,n)).fail(o.reject).progress(a(t,u,s)):--i;return i||o.resolveWith(l,n),o.promise()}}),b.support=function(){var t,n,r,a,s,u,l,c,p,f,d=o.createElement("div");if(d.setAttribute("className","t"),d.innerHTML=" <link/><table></table><a href='/a'>a</a><input type='checkbox'/>",n=d.getElementsByTagName("*"),r=d.getElementsByTagName("a")[0],!n||!r||!n.length)return{};s=o.createElement("select"),l=s.appendChild(o.createElement("option")),a=d.getElementsByTagName("input")[0],r.style.cssText="top:1px;float:left;opacity:.5",t={getSetAttribute:"t"!==d.className,leadingWhitespace:3===d.firstChild.nodeType,tbody:!d.getElementsByTagName("tbody").length,htmlSerialize:!!d.getElementsByTagName("link").length,style:/top/.test(r.getAttribute("style")),hrefNormalized:"/a"===r.getAttribute("href"),opacity:/^0.5/.test(r.style.opacity),cssFloat:!!r.style.cssFloat,checkOn:!!a.value,optSelected:l.selected,enctype:!!o.createElement("form").enctype,html5Clone:"<:nav></:nav>"!==o.createElement("nav").cloneNode(!0).outerHTML,boxModel:"CSS1Compat"===o.compatMode,deleteExpando:!0,noCloneEvent:!0,inlineBlockNeedsLayout:!1,shrinkWrapBlocks:!1,reliableMarginRight:!0,boxSizingReliable:!0,pixelPosition:!1},a.checked=!0,t.noCloneChecked=a.cloneNode(!0).checked,s.disabled=!0,t.optDisabled=!l.disabled;try{delete d.test}catch(h){t.deleteExpando=!1}a=o.createElement("input"),a.setAttribute("value",""),t.input=""===a.getAttribute("value"),a.value="t",a.setAttribute("type","radio"),t.radioValue="t"===a.value,a.setAttribute("checked","t"),a.setAttribute("name","t"),u=o.createDocumentFragment(),u.appendChild(a),t.appendChecked=a.checked,t.checkClone=u.cloneNode(!0).cloneNode(!0).lastChild.checked,d.attachEvent&&(d.attachEvent("onclick",function(){t.noCloneEvent=!1}),d.cloneNode(!0).click());for(f in{submit:!0,change:!0,focusin:!0})d.setAttribute(c="on"+f,"t"),t[f+"Bubbles"]=c in e||d.attributes[c].expando===!1;return d.style.backgroundClip="content-box",d.cloneNode(!0).style.backgroundClip="",t.clearCloneStyle="content-box"===d.style.backgroundClip,b(function(){var n,r,a,s="padding:0;margin:0;border:0;display:block;box-sizing:content-box;-moz-box-sizing:content-box;-webkit-box-sizing:content-box;",u=o.getElementsByTagName("body")[0];u&&(n=o.createElement("div"),n.style.cssText="border:0;width:0;height:0;position:absolute;top:0;left:-9999px;margin-top:1px",u.appendChild(n).appendChild(d),d.innerHTML="<table><tr><td></td><td>t</td></tr></table>",a=d.getElementsByTagName("td"),a[0].style.cssText="padding:0;margin:0;border:0;display:none",p=0===a[0].offsetHeight,a[0].style.display="",a[1].style.display="none",t.reliableHiddenOffsets=p&&0===a[0].offsetHeight,d.innerHTML="",d.style.cssText="box-sizing:border-box;-moz-box-sizing:border-box;-webkit-box-sizing:border-box;padding:1px;border:1px;display:block;width:4px;margin-top:1%;position:absolute;top:1%;",t.boxSizing=4===d.offsetWidth,t.doesNotIncludeMarginInBodyOffset=1!==u.offsetTop,e.getComputedStyle&&(t.pixelPosition="1%"!==(e.getComputedStyle(d,null)||{}).top,t.boxSizingReliable="4px"===(e.getComputedStyle(d,null)||{width:"4px"}).width,r=d.appendChild(o.createElement("div")),r.style.cssText=d.style.cssText=s,r.style.marginRight=r.style.width="0",d.style.width="1px",t.reliableMarginRight=!parseFloat((e.getComputedStyle(r,null)||{}).marginRight)),typeof d.style.zoom!==i&&(d.innerHTML="",d.style.cssText=s+"width:1px;padding:1px;display:inline;zoom:1",t.inlineBlockNeedsLayout=3===d.offsetWidth,d.style.display="block",d.innerHTML="<div></div>",d.firstChild.style.width="5px",t.shrinkWrapBlocks=3!==d.offsetWidth,t.inlineBlockNeedsLayout&&(u.style.zoom=1)),u.removeChild(n),n=d=a=r=null)}),n=s=u=l=r=a=null,t}();var O=/(?:\{[\s\S]*\}|\[[\s\S]*\])$/,B=/([A-Z])/g;function P(e,n,r,i){if(b.acceptData(e)){var o,a,s=b.expando,u="string"==typeof n,l=e.nodeType,p=l?b.cache:e,f=l?e[s]:e[s]&&s;if(f&&p[f]&&(i||p[f].data)||!u||r!==t)return f||(l?e[s]=f=c.pop()||b.guid++:f=s),p[f]||(p[f]={},l||(p[f].toJSON=b.noop)),("object"==typeof n||"function"==typeof n)&&(i?p[f]=b.extend(p[f],n):p[f].data=b.extend(p[f].data,n)),o=p[f],i||(o.data||(o.data={}),o=o.data),r!==t&&(o[b.camelCase(n)]=r),u?(a=o[n],null==a&&(a=o[b.camelCase(n)])):a=o,a}}function R(e,t,n){if(b.acceptData(e)){var r,i,o,a=e.nodeType,s=a?b.cache:e,u=a?e[b.expando]:b.expando;if(s[u]){if(t&&(o=n?s[u]:s[u].data)){b.isArray(t)?t=t.concat(b.map(t,b.camelCase)):t in o?t=[t]:(t=b.camelCase(t),t=t in o?[t]:t.split(" "));for(r=0,i=t.length;i>r;r++)delete o[t[r]];if(!(n?$:b.isEmptyObject)(o))return}(n||(delete s[u].data,$(s[u])))&&(a?b.cleanData([e],!0):b.support.deleteExpando||s!=s.window?delete s[u]:s[u]=null)}}}b.extend({cache:{},expando:"jQuery"+(p+Math.random()).replace(/\D/g,""),noData:{embed:!0,object:"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000",applet:!0},hasData:function(e){return e=e.nodeType?b.cache[e[b.expando]]:e[b.expando],!!e&&!$(e)},data:function(e,t,n){return P(e,t,n)},removeData:function(e,t){return R(e,t)},_data:function(e,t,n){return P(e,t,n,!0)},_removeData:function(e,t){return R(e,t,!0)},acceptData:function(e){if(e.nodeType&&1!==e.nodeType&&9!==e.nodeType)return!1;var t=e.nodeName&&b.noData[e.nodeName.toLowerCase()];return!t||t!==!0&&e.getAttribute("classid")===t}}),b.fn.extend({data:function(e,n){var r,i,o=this[0],a=0,s=null;if(e===t){if(this.length&&(s=b.data(o),1===o.nodeType&&!b._data(o,"parsedAttrs"))){for(r=o.attributes;r.length>a;a++)i=r[a].name,i.indexOf("data-")||(i=b.camelCase(i.slice(5)),W(o,i,s[i]));b._data(o,"parsedAttrs",!0)}return s}return"object"==typeof e?this.each(function(){b.data(this,e)}):b.access(this,function(n){return n===t?o?W(o,e,b.data(o,e)):null:(this.each(function(){b.data(this,e,n)}),t)},null,n,arguments.length>1,null,!0)},removeData:function(e){return this.each(function(){b.removeData(this,e)})}});function W(e,n,r){if(r===t&&1===e.nodeType){var i="data-"+n.replace(B,"-$1").toLowerCase();if(r=e.getAttribute(i),"string"==typeof r){try{r="true"===r?!0:"false"===r?!1:"null"===r?null:+r+""===r?+r:O.test(r)?b.parseJSON(r):r}catch(o){}b.data(e,n,r)}else r=t}return r}function $(e){var t;for(t in e)if(("data"!==t||!b.isEmptyObject(e[t]))&&"toJSON"!==t)return!1;return!0}b.extend({queue:function(e,n,r){var i;return e?(n=(n||"fx")+"queue",i=b._data(e,n),r&&(!i||b.isArray(r)?i=b._data(e,n,b.makeArray(r)):i.push(r)),i||[]):t},dequeue:function(e,t){t=t||"fx";var n=b.queue(e,t),r=n.length,i=n.shift(),o=b._queueHooks(e,t),a=function(){b.dequeue(e,t)};"inprogress"===i&&(i=n.shift(),r--),o.cur=i,i&&("fx"===t&&n.unshift("inprogress"),delete o.stop,i.call(e,a,o)),!r&&o&&o.empty.fire()},_queueHooks:function(e,t){var n=t+"queueHooks";return b._data(e,n)||b._data(e,n,{empty:b.Callbacks("once memory").add(function(){b._removeData(e,t+"queue"),b._removeData(e,n)})})}}),b.fn.extend({queue:function(e,n){var r=2;return"string"!=typeof e&&(n=e,e="fx",r--),r>arguments.length?b.queue(this[0],e):n===t?this:this.each(function(){var t=b.queue(this,e,n);b._queueHooks(this,e),"fx"===e&&"inprogress"!==t[0]&&b.dequeue(this,e)})},dequeue:function(e){return this.each(function(){b.dequeue(this,e)})},delay:function(e,t){return e=b.fx?b.fx.speeds[e]||e:e,t=t||"fx",this.queue(t,function(t,n){var r=setTimeout(t,e);n.stop=function(){clearTimeout(r)}})},clearQueue:function(e){return this.queue(e||"fx",[])},promise:function(e,n){var r,i=1,o=b.Deferred(),a=this,s=this.length,u=function(){--i||o.resolveWith(a,[a])};"string"!=typeof e&&(n=e,e=t),e=e||"fx";while(s--)r=b._data(a[s],e+"queueHooks"),r&&r.empty&&(i++,r.empty.add(u));return u(),o.promise(n)}});var I,z,X=/[\t\r\n]/g,U=/\r/g,V=/^(?:input|select|textarea|button|object)$/i,Y=/^(?:a|area)$/i,J=/^(?:checked|selected|autofocus|autoplay|async|controls|defer|disabled|hidden|loop|multiple|open|readonly|required|scoped)$/i,G=/^(?:checked|selected)$/i,Q=b.support.getSetAttribute,K=b.support.input;b.fn.extend({attr:function(e,t){return b.access(this,b.attr,e,t,arguments.length>1)},removeAttr:function(e){return this.each(function(){b.removeAttr(this,e)})},prop:function(e,t){return b.access(this,b.prop,e,t,arguments.length>1)},removeProp:function(e){return e=b.propFix[e]||e,this.each(function(){try{this[e]=t,delete this[e]}catch(n){}})},addClass:function(e){var t,n,r,i,o,a=0,s=this.length,u="string"==typeof e&&e;if(b.isFunction(e))return this.each(function(t){b(this).addClass(e.call(this,t,this.className))});if(u)for(t=(e||"").match(w)||[];s>a;a++)if(n=this[a],r=1===n.nodeType&&(n.className?(" "+n.className+" ").replace(X," "):" ")){o=0;while(i=t[o++])0>r.indexOf(" "+i+" ")&&(r+=i+" ");n.className=b.trim(r)}return this},removeClass:function(e){var t,n,r,i,o,a=0,s=this.length,u=0===arguments.length||"string"==typeof e&&e;if(b.isFunction(e))return this.each(function(t){b(this).removeClass(e.call(this,t,this.className))});if(u)for(t=(e||"").match(w)||[];s>a;a++)if(n=this[a],r=1===n.nodeType&&(n.className?(" "+n.className+" ").replace(X," "):"")){o=0;while(i=t[o++])while(r.indexOf(" "+i+" ")>=0)r=r.replace(" "+i+" "," ");n.className=e?b.trim(r):""}return this},toggleClass:function(e,t){var n=typeof e,r="boolean"==typeof t;return b.isFunction(e)?this.each(function(n){b(this).toggleClass(e.call(this,n,this.className,t),t)}):this.each(function(){if("string"===n){var o,a=0,s=b(this),u=t,l=e.match(w)||[];while(o=l[a++])u=r?u:!s.hasClass(o),s[u?"addClass":"removeClass"](o)}else(n===i||"boolean"===n)&&(this.className&&b._data(this,"__className__",this.className),this.className=this.className||e===!1?"":b._data(this,"__className__")||"")})},hasClass:function(e){var t=" "+e+" ",n=0,r=this.length;for(;r>n;n++)if(1===this[n].nodeType&&(" "+this[n].className+" ").replace(X," ").indexOf(t)>=0)return!0;return!1},val:function(e){var n,r,i,o=this[0];{if(arguments.length)return i=b.isFunction(e),this.each(function(n){var o,a=b(this);1===this.nodeType&&(o=i?e.call(this,n,a.val()):e,null==o?o="":"number"==typeof o?o+="":b.isArray(o)&&(o=b.map(o,function(e){return null==e?"":e+""})),r=b.valHooks[this.type]||b.valHooks[this.nodeName.toLowerCase()],r&&"set"in r&&r.set(this,o,"value")!==t||(this.value=o))});if(o)return r=b.valHooks[o.type]||b.valHooks[o.nodeName.toLowerCase()],r&&"get"in r&&(n=r.get(o,"value"))!==t?n:(n=o.value,"string"==typeof n?n.replace(U,""):null==n?"":n)}}}),b.extend({valHooks:{option:{get:function(e){var t=e.attributes.value;return!t||t.specified?e.value:e.text}},select:{get:function(e){var t,n,r=e.options,i=e.selectedIndex,o="select-one"===e.type||0>i,a=o?null:[],s=o?i+1:r.length,u=0>i?s:o?i:0;for(;s>u;u++)if(n=r[u],!(!n.selected&&u!==i||(b.support.optDisabled?n.disabled:null!==n.getAttribute("disabled"))||n.parentNode.disabled&&b.nodeName(n.parentNode,"optgroup"))){if(t=b(n).val(),o)return t;a.push(t)}return a},set:function(e,t){var n=b.makeArray(t);return b(e).find("option").each(function(){this.selected=b.inArray(b(this).val(),n)>=0}),n.length||(e.selectedIndex=-1),n}}},attr:function(e,n,r){var o,a,s,u=e.nodeType;if(e&&3!==u&&8!==u&&2!==u)return typeof e.getAttribute===i?b.prop(e,n,r):(a=1!==u||!b.isXMLDoc(e),a&&(n=n.toLowerCase(),o=b.attrHooks[n]||(J.test(n)?z:I)),r===t?o&&a&&"get"in o&&null!==(s=o.get(e,n))?s:(typeof e.getAttribute!==i&&(s=e.getAttribute(n)),null==s?t:s):null!==r?o&&a&&"set"in o&&(s=o.set(e,r,n))!==t?s:(e.setAttribute(n,r+""),r):(b.removeAttr(e,n),t))},removeAttr:function(e,t){var n,r,i=0,o=t&&t.match(w);if(o&&1===e.nodeType)while(n=o[i++])r=b.propFix[n]||n,J.test(n)?!Q&&G.test(n)?e[b.camelCase("default-"+n)]=e[r]=!1:e[r]=!1:b.attr(e,n,""),e.removeAttribute(Q?n:r)},attrHooks:{type:{set:function(e,t){if(!b.support.radioValue&&"radio"===t&&b.nodeName(e,"input")){var n=e.value;return e.setAttribute("type",t),n&&(e.value=n),t}}}},propFix:{tabindex:"tabIndex",readonly:"readOnly","for":"htmlFor","class":"className",maxlength:"maxLength",cellspacing:"cellSpacing",cellpadding:"cellPadding",rowspan:"rowSpan",colspan:"colSpan",usemap:"useMap",frameborder:"frameBorder",contenteditable:"contentEditable"},prop:function(e,n,r){var i,o,a,s=e.nodeType;if(e&&3!==s&&8!==s&&2!==s)return a=1!==s||!b.isXMLDoc(e),a&&(n=b.propFix[n]||n,o=b.propHooks[n]),r!==t?o&&"set"in o&&(i=o.set(e,r,n))!==t?i:e[n]=r:o&&"get"in o&&null!==(i=o.get(e,n))?i:e[n]},propHooks:{tabIndex:{get:function(e){var n=e.getAttributeNode("tabindex");return n&&n.specified?parseInt(n.value,10):V.test(e.nodeName)||Y.test(e.nodeName)&&e.href?0:t}}}}),z={get:function(e,n){var r=b.prop(e,n),i="boolean"==typeof r&&e.getAttribute(n),o="boolean"==typeof r?K&&Q?null!=i:G.test(n)?e[b.camelCase("default-"+n)]:!!i:e.getAttributeNode(n);return o&&o.value!==!1?n.toLowerCase():t},set:function(e,t,n){return t===!1?b.removeAttr(e,n):K&&Q||!G.test(n)?e.setAttribute(!Q&&b.propFix[n]||n,n):e[b.camelCase("default-"+n)]=e[n]=!0,n}},K&&Q||(b.attrHooks.value={get:function(e,n){var r=e.getAttributeNode(n);return b.nodeName(e,"input")?e.defaultValue:r&&r.specified?r.value:t},set:function(e,n,r){return b.nodeName(e,"input")?(e.defaultValue=n,t):I&&I.set(e,n,r)}}),Q||(I=b.valHooks.button={get:function(e,n){var r=e.getAttributeNode(n);return r&&("id"===n||"name"===n||"coords"===n?""!==r.value:r.specified)?r.value:t},set:function(e,n,r){var i=e.getAttributeNode(r);return i||e.setAttributeNode(i=e.ownerDocument.createAttribute(r)),i.value=n+="","value"===r||n===e.getAttribute(r)?n:t}},b.attrHooks.contenteditable={get:I.get,set:function(e,t,n){I.set(e,""===t?!1:t,n)}},b.each(["width","height"],function(e,n){b.attrHooks[n]=b.extend(b.attrHooks[n],{set:function(e,r){return""===r?(e.setAttribute(n,"auto"),r):t}})})),b.support.hrefNormalized||(b.each(["href","src","width","height"],function(e,n){b.attrHooks[n]=b.extend(b.attrHooks[n],{get:function(e){var r=e.getAttribute(n,2);return null==r?t:r}})}),b.each(["href","src"],function(e,t){b.propHooks[t]={get:function(e){return e.getAttribute(t,4)}}})),b.support.style||(b.attrHooks.style={get:function(e){return e.style.cssText||t},set:function(e,t){return e.style.cssText=t+""}}),b.support.optSelected||(b.propHooks.selected=b.extend(b.propHooks.selected,{get:function(e){var t=e.parentNode;return t&&(t.selectedIndex,t.parentNode&&t.parentNode.selectedIndex),null}})),b.support.enctype||(b.propFix.enctype="encoding"),b.support.checkOn||b.each(["radio","checkbox"],function(){b.valHooks[this]={get:function(e){return null===e.getAttribute("value")?"on":e.value}}}),b.each(["radio","checkbox"],function(){b.valHooks[this]=b.extend(b.valHooks[this],{set:function(e,n){return b.isArray(n)?e.checked=b.inArray(b(e).val(),n)>=0:t}})});var Z=/^(?:input|select|textarea)$/i,et=/^key/,tt=/^(?:mouse|contextmenu)|click/,nt=/^(?:focusinfocus|focusoutblur)$/,rt=/^([^.]*)(?:\.(.+)|)$/;function it(){return!0}function ot(){return!1}b.event={global:{},add:function(e,n,r,o,a){var s,u,l,c,p,f,d,h,g,m,y,v=b._data(e);if(v){r.handler&&(c=r,r=c.handler,a=c.selector),r.guid||(r.guid=b.guid++),(u=v.events)||(u=v.events={}),(f=v.handle)||(f=v.handle=function(e){return typeof b===i||e&&b.event.triggered===e.type?t:b.event.dispatch.apply(f.elem,arguments)},f.elem=e),n=(n||"").match(w)||[""],l=n.length;while(l--)s=rt.exec(n[l])||[],g=y=s[1],m=(s[2]||"").split(".").sort(),p=b.event.special[g]||{},g=(a?p.delegateType:p.bindType)||g,p=b.event.special[g]||{},d=b.extend({type:g,origType:y,data:o,handler:r,guid:r.guid,selector:a,needsContext:a&&b.expr.match.needsContext.test(a),namespace:m.join(".")},c),(h=u[g])||(h=u[g]=[],h.delegateCount=0,p.setup&&p.setup.call(e,o,m,f)!==!1||(e.addEventListener?e.addEventListener(g,f,!1):e.attachEvent&&e.attachEvent("on"+g,f))),p.add&&(p.add.call(e,d),d.handler.guid||(d.handler.guid=r.guid)),a?h.splice(h.delegateCount++,0,d):h.push(d),b.event.global[g]=!0;e=null}},remove:function(e,t,n,r,i){var o,a,s,u,l,c,p,f,d,h,g,m=b.hasData(e)&&b._data(e);if(m&&(c=m.events)){t=(t||"").match(w)||[""],l=t.length;while(l--)if(s=rt.exec(t[l])||[],d=g=s[1],h=(s[2]||"").split(".").sort(),d){p=b.event.special[d]||{},d=(r?p.delegateType:p.bindType)||d,f=c[d]||[],s=s[2]&&RegExp("(^|\\.)"+h.join("\\.(?:.*\\.|)")+"(\\.|$)"),u=o=f.length;while(o--)a=f[o],!i&&g!==a.origType||n&&n.guid!==a.guid||s&&!s.test(a.namespace)||r&&r!==a.selector&&("**"!==r||!a.selector)||(f.splice(o,1),a.selector&&f.delegateCount--,p.remove&&p.remove.call(e,a));u&&!f.length&&(p.teardown&&p.teardown.call(e,h,m.handle)!==!1||b.removeEvent(e,d,m.handle),delete c[d])}else for(d in c)b.event.remove(e,d+t[l],n,r,!0);b.isEmptyObject(c)&&(delete m.handle,b._removeData(e,"events"))}},trigger:function(n,r,i,a){var s,u,l,c,p,f,d,h=[i||o],g=y.call(n,"type")?n.type:n,m=y.call(n,"namespace")?n.namespace.split("."):[];if(l=f=i=i||o,3!==i.nodeType&&8!==i.nodeType&&!nt.test(g+b.event.triggered)&&(g.indexOf(".")>=0&&(m=g.split("."),g=m.shift(),m.sort()),u=0>g.indexOf(":")&&"on"+g,n=n[b.expando]?n:new b.Event(g,"object"==typeof n&&n),n.isTrigger=!0,n.namespace=m.join("."),n.namespace_re=n.namespace?RegExp("(^|\\.)"+m.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,n.result=t,n.target||(n.target=i),r=null==r?[n]:b.makeArray(r,[n]),p=b.event.special[g]||{},a||!p.trigger||p.trigger.apply(i,r)!==!1)){if(!a&&!p.noBubble&&!b.isWindow(i)){for(c=p.delegateType||g,nt.test(c+g)||(l=l.parentNode);l;l=l.parentNode)h.push(l),f=l;f===(i.ownerDocument||o)&&h.push(f.defaultView||f.parentWindow||e)}d=0;while((l=h[d++])&&!n.isPropagationStopped())n.type=d>1?c:p.bindType||g,s=(b._data(l,"events")||{})[n.type]&&b._data(l,"handle"),s&&s.apply(l,r),s=u&&l[u],s&&b.acceptData(l)&&s.apply&&s.apply(l,r)===!1&&n.preventDefault();if(n.type=g,!(a||n.isDefaultPrevented()||p._default&&p._default.apply(i.ownerDocument,r)!==!1||"click"===g&&b.nodeName(i,"a")||!b.acceptData(i)||!u||!i[g]||b.isWindow(i))){f=i[u],f&&(i[u]=null),b.event.triggered=g;try{i[g]()}catch(v){}b.event.triggered=t,f&&(i[u]=f)}return n.result}},dispatch:function(e){e=b.event.fix(e);var n,r,i,o,a,s=[],u=h.call(arguments),l=(b._data(this,"events")||{})[e.type]||[],c=b.event.special[e.type]||{};if(u[0]=e,e.delegateTarget=this,!c.preDispatch||c.preDispatch.call(this,e)!==!1){s=b.event.handlers.call(this,e,l),n=0;while((o=s[n++])&&!e.isPropagationStopped()){e.currentTarget=o.elem,a=0;while((i=o.handlers[a++])&&!e.isImmediatePropagationStopped())(!e.namespace_re||e.namespace_re.test(i.namespace))&&(e.handleObj=i,e.data=i.data,r=((b.event.special[i.origType]||{}).handle||i.handler).apply(o.elem,u),r!==t&&(e.result=r)===!1&&(e.preventDefault(),e.stopPropagation()))}return c.postDispatch&&c.postDispatch.call(this,e),e.result}},handlers:function(e,n){var r,i,o,a,s=[],u=n.delegateCount,l=e.target;if(u&&l.nodeType&&(!e.button||"click"!==e.type))for(;l!=this;l=l.parentNode||this)if(1===l.nodeType&&(l.disabled!==!0||"click"!==e.type)){for(o=[],a=0;u>a;a++)i=n[a],r=i.selector+" ",o[r]===t&&(o[r]=i.needsContext?b(r,this).index(l)>=0:b.find(r,this,null,[l]).length),o[r]&&o.push(i);o.length&&s.push({elem:l,handlers:o})}return n.length>u&&s.push({elem:this,handlers:n.slice(u)}),s},fix:function(e){if(e[b.expando])return e;var t,n,r,i=e.type,a=e,s=this.fixHooks[i];s||(this.fixHooks[i]=s=tt.test(i)?this.mouseHooks:et.test(i)?this.keyHooks:{}),r=s.props?this.props.concat(s.props):this.props,e=new b.Event(a),t=r.length;while(t--)n=r[t],e[n]=a[n];return e.target||(e.target=a.srcElement||o),3===e.target.nodeType&&(e.target=e.target.parentNode),e.metaKey=!!e.metaKey,s.filter?s.filter(e,a):e},props:"altKey bubbles cancelable ctrlKey currentTarget eventPhase metaKey relatedTarget shiftKey target timeStamp view which".split(" "),fixHooks:{},keyHooks:{props:"char charCode key keyCode".split(" "),filter:function(e,t){return null==e.which&&(e.which=null!=t.charCode?t.charCode:t.keyCode),e}},mouseHooks:{props:"button buttons clientX clientY fromElement offsetX offsetY pageX pageY screenX screenY toElement".split(" "),filter:function(e,n){var r,i,a,s=n.button,u=n.fromElement;return null==e.pageX&&null!=n.clientX&&(i=e.target.ownerDocument||o,a=i.documentElement,r=i.body,e.pageX=n.clientX+(a&&a.scrollLeft||r&&r.scrollLeft||0)-(a&&a.clientLeft||r&&r.clientLeft||0),e.pageY=n.clientY+(a&&a.scrollTop||r&&r.scrollTop||0)-(a&&a.clientTop||r&&r.clientTop||0)),!e.relatedTarget&&u&&(e.relatedTarget=u===e.target?n.toElement:u),e.which||s===t||(e.which=1&s?1:2&s?3:4&s?2:0),e}},special:{load:{noBubble:!0},click:{trigger:function(){return b.nodeName(this,"input")&&"checkbox"===this.type&&this.click?(this.click(),!1):t}},focus:{trigger:function(){if(this!==o.activeElement&&this.focus)try{return this.focus(),!1}catch(e){}},delegateType:"focusin"},blur:{trigger:function(){return this===o.activeElement&&this.blur?(this.blur(),!1):t},delegateType:"focusout"},beforeunload:{postDispatch:function(e){e.result!==t&&(e.originalEvent.returnValue=e.result)}}},simulate:function(e,t,n,r){var i=b.extend(new b.Event,n,{type:e,isSimulated:!0,originalEvent:{}});r?b.event.trigger(i,null,t):b.event.dispatch.call(t,i),i.isDefaultPrevented()&&n.preventDefault()}},b.removeEvent=o.removeEventListener?function(e,t,n){e.removeEventListener&&e.removeEventListener(t,n,!1)}:function(e,t,n){var r="on"+t;e.detachEvent&&(typeof e[r]===i&&(e[r]=null),e.detachEvent(r,n))},b.Event=function(e,n){return this instanceof b.Event?(e&&e.type?(this.originalEvent=e,this.type=e.type,this.isDefaultPrevented=e.defaultPrevented||e.returnValue===!1||e.getPreventDefault&&e.getPreventDefault()?it:ot):this.type=e,n&&b.extend(this,n),this.timeStamp=e&&e.timeStamp||b.now(),this[b.expando]=!0,t):new b.Event(e,n)},b.Event.prototype={isDefaultPrevented:ot,isPropagationStopped:ot,isImmediatePropagationStopped:ot,preventDefault:function(){var e=this.originalEvent;this.isDefaultPrevented=it,e&&(e.preventDefault?e.preventDefault():e.returnValue=!1)},stopPropagation:function(){var e=this.originalEvent;this.isPropagationStopped=it,e&&(e.stopPropagation&&e.stopPropagation(),e.cancelBubble=!0)},stopImmediatePropagation:function(){this.isImmediatePropagationStopped=it,this.stopPropagation()}},b.each({mouseenter:"mouseover",mouseleave:"mouseout"},function(e,t){b.event.special[e]={delegateType:t,bindType:t,handle:function(e){var n,r=this,i=e.relatedTarget,o=e.handleObj;
return(!i||i!==r&&!b.contains(r,i))&&(e.type=o.origType,n=o.handler.apply(this,arguments),e.type=t),n}}}),b.support.submitBubbles||(b.event.special.submit={setup:function(){return b.nodeName(this,"form")?!1:(b.event.add(this,"click._submit keypress._submit",function(e){var n=e.target,r=b.nodeName(n,"input")||b.nodeName(n,"button")?n.form:t;r&&!b._data(r,"submitBubbles")&&(b.event.add(r,"submit._submit",function(e){e._submit_bubble=!0}),b._data(r,"submitBubbles",!0))}),t)},postDispatch:function(e){e._submit_bubble&&(delete e._submit_bubble,this.parentNode&&!e.isTrigger&&b.event.simulate("submit",this.parentNode,e,!0))},teardown:function(){return b.nodeName(this,"form")?!1:(b.event.remove(this,"._submit"),t)}}),b.support.changeBubbles||(b.event.special.change={setup:function(){return Z.test(this.nodeName)?(("checkbox"===this.type||"radio"===this.type)&&(b.event.add(this,"propertychange._change",function(e){"checked"===e.originalEvent.propertyName&&(this._just_changed=!0)}),b.event.add(this,"click._change",function(e){this._just_changed&&!e.isTrigger&&(this._just_changed=!1),b.event.simulate("change",this,e,!0)})),!1):(b.event.add(this,"beforeactivate._change",function(e){var t=e.target;Z.test(t.nodeName)&&!b._data(t,"changeBubbles")&&(b.event.add(t,"change._change",function(e){!this.parentNode||e.isSimulated||e.isTrigger||b.event.simulate("change",this.parentNode,e,!0)}),b._data(t,"changeBubbles",!0))}),t)},handle:function(e){var n=e.target;return this!==n||e.isSimulated||e.isTrigger||"radio"!==n.type&&"checkbox"!==n.type?e.handleObj.handler.apply(this,arguments):t},teardown:function(){return b.event.remove(this,"._change"),!Z.test(this.nodeName)}}),b.support.focusinBubbles||b.each({focus:"focusin",blur:"focusout"},function(e,t){var n=0,r=function(e){b.event.simulate(t,e.target,b.event.fix(e),!0)};b.event.special[t]={setup:function(){0===n++&&o.addEventListener(e,r,!0)},teardown:function(){0===--n&&o.removeEventListener(e,r,!0)}}}),b.fn.extend({on:function(e,n,r,i,o){var a,s;if("object"==typeof e){"string"!=typeof n&&(r=r||n,n=t);for(a in e)this.on(a,n,r,e[a],o);return this}if(null==r&&null==i?(i=n,r=n=t):null==i&&("string"==typeof n?(i=r,r=t):(i=r,r=n,n=t)),i===!1)i=ot;else if(!i)return this;return 1===o&&(s=i,i=function(e){return b().off(e),s.apply(this,arguments)},i.guid=s.guid||(s.guid=b.guid++)),this.each(function(){b.event.add(this,e,i,r,n)})},one:function(e,t,n,r){return this.on(e,t,n,r,1)},off:function(e,n,r){var i,o;if(e&&e.preventDefault&&e.handleObj)return i=e.handleObj,b(e.delegateTarget).off(i.namespace?i.origType+"."+i.namespace:i.origType,i.selector,i.handler),this;if("object"==typeof e){for(o in e)this.off(o,n,e[o]);return this}return(n===!1||"function"==typeof n)&&(r=n,n=t),r===!1&&(r=ot),this.each(function(){b.event.remove(this,e,r,n)})},bind:function(e,t,n){return this.on(e,null,t,n)},unbind:function(e,t){return this.off(e,null,t)},delegate:function(e,t,n,r){return this.on(t,e,n,r)},undelegate:function(e,t,n){return 1===arguments.length?this.off(e,"**"):this.off(t,e||"**",n)},trigger:function(e,t){return this.each(function(){b.event.trigger(e,t,this)})},triggerHandler:function(e,n){var r=this[0];return r?b.event.trigger(e,n,r,!0):t}}),function(e,t){var n,r,i,o,a,s,u,l,c,p,f,d,h,g,m,y,v,x="sizzle"+-new Date,w=e.document,T={},N=0,C=0,k=it(),E=it(),S=it(),A=typeof t,j=1<<31,D=[],L=D.pop,H=D.push,q=D.slice,M=D.indexOf||function(e){var t=0,n=this.length;for(;n>t;t++)if(this[t]===e)return t;return-1},_="[\\x20\\t\\r\\n\\f]",F="(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",O=F.replace("w","w#"),B="([*^$|!~]?=)",P="\\["+_+"*("+F+")"+_+"*(?:"+B+_+"*(?:(['\"])((?:\\\\.|[^\\\\])*?)\\3|("+O+")|)|)"+_+"*\\]",R=":("+F+")(?:\\(((['\"])((?:\\\\.|[^\\\\])*?)\\3|((?:\\\\.|[^\\\\()[\\]]|"+P.replace(3,8)+")*)|.*)\\)|)",W=RegExp("^"+_+"+|((?:^|[^\\\\])(?:\\\\.)*)"+_+"+$","g"),$=RegExp("^"+_+"*,"+_+"*"),I=RegExp("^"+_+"*([\\x20\\t\\r\\n\\f>+~])"+_+"*"),z=RegExp(R),X=RegExp("^"+O+"$"),U={ID:RegExp("^#("+F+")"),CLASS:RegExp("^\\.("+F+")"),NAME:RegExp("^\\[name=['\"]?("+F+")['\"]?\\]"),TAG:RegExp("^("+F.replace("w","w*")+")"),ATTR:RegExp("^"+P),PSEUDO:RegExp("^"+R),CHILD:RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+_+"*(even|odd|(([+-]|)(\\d*)n|)"+_+"*(?:([+-]|)"+_+"*(\\d+)|))"+_+"*\\)|)","i"),needsContext:RegExp("^"+_+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+_+"*((?:-\\d)?\\d*)"+_+"*\\)|)(?=[^-]|$)","i")},V=/[\x20\t\r\n\f]*[+~]/,Y=/^[^{]+\{\s*\[native code/,J=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,G=/^(?:input|select|textarea|button)$/i,Q=/^h\d$/i,K=/'|\\/g,Z=/\=[\x20\t\r\n\f]*([^'"\]]*)[\x20\t\r\n\f]*\]/g,et=/\\([\da-fA-F]{1,6}[\x20\t\r\n\f]?|.)/g,tt=function(e,t){var n="0x"+t-65536;return n!==n?t:0>n?String.fromCharCode(n+65536):String.fromCharCode(55296|n>>10,56320|1023&n)};try{q.call(w.documentElement.childNodes,0)[0].nodeType}catch(nt){q=function(e){var t,n=[];while(t=this[e++])n.push(t);return n}}function rt(e){return Y.test(e+"")}function it(){var e,t=[];return e=function(n,r){return t.push(n+=" ")>i.cacheLength&&delete e[t.shift()],e[n]=r}}function ot(e){return e[x]=!0,e}function at(e){var t=p.createElement("div");try{return e(t)}catch(n){return!1}finally{t=null}}function st(e,t,n,r){var i,o,a,s,u,l,f,g,m,v;if((t?t.ownerDocument||t:w)!==p&&c(t),t=t||p,n=n||[],!e||"string"!=typeof e)return n;if(1!==(s=t.nodeType)&&9!==s)return[];if(!d&&!r){if(i=J.exec(e))if(a=i[1]){if(9===s){if(o=t.getElementById(a),!o||!o.parentNode)return n;if(o.id===a)return n.push(o),n}else if(t.ownerDocument&&(o=t.ownerDocument.getElementById(a))&&y(t,o)&&o.id===a)return n.push(o),n}else{if(i[2])return H.apply(n,q.call(t.getElementsByTagName(e),0)),n;if((a=i[3])&&T.getByClassName&&t.getElementsByClassName)return H.apply(n,q.call(t.getElementsByClassName(a),0)),n}if(T.qsa&&!h.test(e)){if(f=!0,g=x,m=t,v=9===s&&e,1===s&&"object"!==t.nodeName.toLowerCase()){l=ft(e),(f=t.getAttribute("id"))?g=f.replace(K,"\\$&"):t.setAttribute("id",g),g="[id='"+g+"'] ",u=l.length;while(u--)l[u]=g+dt(l[u]);m=V.test(e)&&t.parentNode||t,v=l.join(",")}if(v)try{return H.apply(n,q.call(m.querySelectorAll(v),0)),n}catch(b){}finally{f||t.removeAttribute("id")}}}return wt(e.replace(W,"$1"),t,n,r)}a=st.isXML=function(e){var t=e&&(e.ownerDocument||e).documentElement;return t?"HTML"!==t.nodeName:!1},c=st.setDocument=function(e){var n=e?e.ownerDocument||e:w;return n!==p&&9===n.nodeType&&n.documentElement?(p=n,f=n.documentElement,d=a(n),T.tagNameNoComments=at(function(e){return e.appendChild(n.createComment("")),!e.getElementsByTagName("*").length}),T.attributes=at(function(e){e.innerHTML="<select></select>";var t=typeof e.lastChild.getAttribute("multiple");return"boolean"!==t&&"string"!==t}),T.getByClassName=at(function(e){return e.innerHTML="<div class='hidden e'></div><div class='hidden'></div>",e.getElementsByClassName&&e.getElementsByClassName("e").length?(e.lastChild.className="e",2===e.getElementsByClassName("e").length):!1}),T.getByName=at(function(e){e.id=x+0,e.innerHTML="<a name='"+x+"'></a><div name='"+x+"'></div>",f.insertBefore(e,f.firstChild);var t=n.getElementsByName&&n.getElementsByName(x).length===2+n.getElementsByName(x+0).length;return T.getIdNotName=!n.getElementById(x),f.removeChild(e),t}),i.attrHandle=at(function(e){return e.innerHTML="<a href='#'></a>",e.firstChild&&typeof e.firstChild.getAttribute!==A&&"#"===e.firstChild.getAttribute("href")})?{}:{href:function(e){return e.getAttribute("href",2)},type:function(e){return e.getAttribute("type")}},T.getIdNotName?(i.find.ID=function(e,t){if(typeof t.getElementById!==A&&!d){var n=t.getElementById(e);return n&&n.parentNode?[n]:[]}},i.filter.ID=function(e){var t=e.replace(et,tt);return function(e){return e.getAttribute("id")===t}}):(i.find.ID=function(e,n){if(typeof n.getElementById!==A&&!d){var r=n.getElementById(e);return r?r.id===e||typeof r.getAttributeNode!==A&&r.getAttributeNode("id").value===e?[r]:t:[]}},i.filter.ID=function(e){var t=e.replace(et,tt);return function(e){var n=typeof e.getAttributeNode!==A&&e.getAttributeNode("id");return n&&n.value===t}}),i.find.TAG=T.tagNameNoComments?function(e,n){return typeof n.getElementsByTagName!==A?n.getElementsByTagName(e):t}:function(e,t){var n,r=[],i=0,o=t.getElementsByTagName(e);if("*"===e){while(n=o[i++])1===n.nodeType&&r.push(n);return r}return o},i.find.NAME=T.getByName&&function(e,n){return typeof n.getElementsByName!==A?n.getElementsByName(name):t},i.find.CLASS=T.getByClassName&&function(e,n){return typeof n.getElementsByClassName===A||d?t:n.getElementsByClassName(e)},g=[],h=[":focus"],(T.qsa=rt(n.querySelectorAll))&&(at(function(e){e.innerHTML="<select><option selected=''></option></select>",e.querySelectorAll("[selected]").length||h.push("\\["+_+"*(?:checked|disabled|ismap|multiple|readonly|selected|value)"),e.querySelectorAll(":checked").length||h.push(":checked")}),at(function(e){e.innerHTML="<input type='hidden' i=''/>",e.querySelectorAll("[i^='']").length&&h.push("[*^$]="+_+"*(?:\"\"|'')"),e.querySelectorAll(":enabled").length||h.push(":enabled",":disabled"),e.querySelectorAll("*,:x"),h.push(",.*:")})),(T.matchesSelector=rt(m=f.matchesSelector||f.mozMatchesSelector||f.webkitMatchesSelector||f.oMatchesSelector||f.msMatchesSelector))&&at(function(e){T.disconnectedMatch=m.call(e,"div"),m.call(e,"[s!='']:x"),g.push("!=",R)}),h=RegExp(h.join("|")),g=RegExp(g.join("|")),y=rt(f.contains)||f.compareDocumentPosition?function(e,t){var n=9===e.nodeType?e.documentElement:e,r=t&&t.parentNode;return e===r||!(!r||1!==r.nodeType||!(n.contains?n.contains(r):e.compareDocumentPosition&&16&e.compareDocumentPosition(r)))}:function(e,t){if(t)while(t=t.parentNode)if(t===e)return!0;return!1},v=f.compareDocumentPosition?function(e,t){var r;return e===t?(u=!0,0):(r=t.compareDocumentPosition&&e.compareDocumentPosition&&e.compareDocumentPosition(t))?1&r||e.parentNode&&11===e.parentNode.nodeType?e===n||y(w,e)?-1:t===n||y(w,t)?1:0:4&r?-1:1:e.compareDocumentPosition?-1:1}:function(e,t){var r,i=0,o=e.parentNode,a=t.parentNode,s=[e],l=[t];if(e===t)return u=!0,0;if(!o||!a)return e===n?-1:t===n?1:o?-1:a?1:0;if(o===a)return ut(e,t);r=e;while(r=r.parentNode)s.unshift(r);r=t;while(r=r.parentNode)l.unshift(r);while(s[i]===l[i])i++;return i?ut(s[i],l[i]):s[i]===w?-1:l[i]===w?1:0},u=!1,[0,0].sort(v),T.detectDuplicates=u,p):p},st.matches=function(e,t){return st(e,null,null,t)},st.matchesSelector=function(e,t){if((e.ownerDocument||e)!==p&&c(e),t=t.replace(Z,"='$1']"),!(!T.matchesSelector||d||g&&g.test(t)||h.test(t)))try{var n=m.call(e,t);if(n||T.disconnectedMatch||e.document&&11!==e.document.nodeType)return n}catch(r){}return st(t,p,null,[e]).length>0},st.contains=function(e,t){return(e.ownerDocument||e)!==p&&c(e),y(e,t)},st.attr=function(e,t){var n;return(e.ownerDocument||e)!==p&&c(e),d||(t=t.toLowerCase()),(n=i.attrHandle[t])?n(e):d||T.attributes?e.getAttribute(t):((n=e.getAttributeNode(t))||e.getAttribute(t))&&e[t]===!0?t:n&&n.specified?n.value:null},st.error=function(e){throw Error("Syntax error, unrecognized expression: "+e)},st.uniqueSort=function(e){var t,n=[],r=1,i=0;if(u=!T.detectDuplicates,e.sort(v),u){for(;t=e[r];r++)t===e[r-1]&&(i=n.push(r));while(i--)e.splice(n[i],1)}return e};function ut(e,t){var n=t&&e,r=n&&(~t.sourceIndex||j)-(~e.sourceIndex||j);if(r)return r;if(n)while(n=n.nextSibling)if(n===t)return-1;return e?1:-1}function lt(e){return function(t){var n=t.nodeName.toLowerCase();return"input"===n&&t.type===e}}function ct(e){return function(t){var n=t.nodeName.toLowerCase();return("input"===n||"button"===n)&&t.type===e}}function pt(e){return ot(function(t){return t=+t,ot(function(n,r){var i,o=e([],n.length,t),a=o.length;while(a--)n[i=o[a]]&&(n[i]=!(r[i]=n[i]))})})}o=st.getText=function(e){var t,n="",r=0,i=e.nodeType;if(i){if(1===i||9===i||11===i){if("string"==typeof e.textContent)return e.textContent;for(e=e.firstChild;e;e=e.nextSibling)n+=o(e)}else if(3===i||4===i)return e.nodeValue}else for(;t=e[r];r++)n+=o(t);return n},i=st.selectors={cacheLength:50,createPseudo:ot,match:U,find:{},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(e){return e[1]=e[1].replace(et,tt),e[3]=(e[4]||e[5]||"").replace(et,tt),"~="===e[2]&&(e[3]=" "+e[3]+" "),e.slice(0,4)},CHILD:function(e){return e[1]=e[1].toLowerCase(),"nth"===e[1].slice(0,3)?(e[3]||st.error(e[0]),e[4]=+(e[4]?e[5]+(e[6]||1):2*("even"===e[3]||"odd"===e[3])),e[5]=+(e[7]+e[8]||"odd"===e[3])):e[3]&&st.error(e[0]),e},PSEUDO:function(e){var t,n=!e[5]&&e[2];return U.CHILD.test(e[0])?null:(e[4]?e[2]=e[4]:n&&z.test(n)&&(t=ft(n,!0))&&(t=n.indexOf(")",n.length-t)-n.length)&&(e[0]=e[0].slice(0,t),e[2]=n.slice(0,t)),e.slice(0,3))}},filter:{TAG:function(e){return"*"===e?function(){return!0}:(e=e.replace(et,tt).toLowerCase(),function(t){return t.nodeName&&t.nodeName.toLowerCase()===e})},CLASS:function(e){var t=k[e+" "];return t||(t=RegExp("(^|"+_+")"+e+"("+_+"|$)"))&&k(e,function(e){return t.test(e.className||typeof e.getAttribute!==A&&e.getAttribute("class")||"")})},ATTR:function(e,t,n){return function(r){var i=st.attr(r,e);return null==i?"!="===t:t?(i+="","="===t?i===n:"!="===t?i!==n:"^="===t?n&&0===i.indexOf(n):"*="===t?n&&i.indexOf(n)>-1:"$="===t?n&&i.slice(-n.length)===n:"~="===t?(" "+i+" ").indexOf(n)>-1:"|="===t?i===n||i.slice(0,n.length+1)===n+"-":!1):!0}},CHILD:function(e,t,n,r,i){var o="nth"!==e.slice(0,3),a="last"!==e.slice(-4),s="of-type"===t;return 1===r&&0===i?function(e){return!!e.parentNode}:function(t,n,u){var l,c,p,f,d,h,g=o!==a?"nextSibling":"previousSibling",m=t.parentNode,y=s&&t.nodeName.toLowerCase(),v=!u&&!s;if(m){if(o){while(g){p=t;while(p=p[g])if(s?p.nodeName.toLowerCase()===y:1===p.nodeType)return!1;h=g="only"===e&&!h&&"nextSibling"}return!0}if(h=[a?m.firstChild:m.lastChild],a&&v){c=m[x]||(m[x]={}),l=c[e]||[],d=l[0]===N&&l[1],f=l[0]===N&&l[2],p=d&&m.childNodes[d];while(p=++d&&p&&p[g]||(f=d=0)||h.pop())if(1===p.nodeType&&++f&&p===t){c[e]=[N,d,f];break}}else if(v&&(l=(t[x]||(t[x]={}))[e])&&l[0]===N)f=l[1];else while(p=++d&&p&&p[g]||(f=d=0)||h.pop())if((s?p.nodeName.toLowerCase()===y:1===p.nodeType)&&++f&&(v&&((p[x]||(p[x]={}))[e]=[N,f]),p===t))break;return f-=i,f===r||0===f%r&&f/r>=0}}},PSEUDO:function(e,t){var n,r=i.pseudos[e]||i.setFilters[e.toLowerCase()]||st.error("unsupported pseudo: "+e);return r[x]?r(t):r.length>1?(n=[e,e,"",t],i.setFilters.hasOwnProperty(e.toLowerCase())?ot(function(e,n){var i,o=r(e,t),a=o.length;while(a--)i=M.call(e,o[a]),e[i]=!(n[i]=o[a])}):function(e){return r(e,0,n)}):r}},pseudos:{not:ot(function(e){var t=[],n=[],r=s(e.replace(W,"$1"));return r[x]?ot(function(e,t,n,i){var o,a=r(e,null,i,[]),s=e.length;while(s--)(o=a[s])&&(e[s]=!(t[s]=o))}):function(e,i,o){return t[0]=e,r(t,null,o,n),!n.pop()}}),has:ot(function(e){return function(t){return st(e,t).length>0}}),contains:ot(function(e){return function(t){return(t.textContent||t.innerText||o(t)).indexOf(e)>-1}}),lang:ot(function(e){return X.test(e||"")||st.error("unsupported lang: "+e),e=e.replace(et,tt).toLowerCase(),function(t){var n;do if(n=d?t.getAttribute("xml:lang")||t.getAttribute("lang"):t.lang)return n=n.toLowerCase(),n===e||0===n.indexOf(e+"-");while((t=t.parentNode)&&1===t.nodeType);return!1}}),target:function(t){var n=e.location&&e.location.hash;return n&&n.slice(1)===t.id},root:function(e){return e===f},focus:function(e){return e===p.activeElement&&(!p.hasFocus||p.hasFocus())&&!!(e.type||e.href||~e.tabIndex)},enabled:function(e){return e.disabled===!1},disabled:function(e){return e.disabled===!0},checked:function(e){var t=e.nodeName.toLowerCase();return"input"===t&&!!e.checked||"option"===t&&!!e.selected},selected:function(e){return e.parentNode&&e.parentNode.selectedIndex,e.selected===!0},empty:function(e){for(e=e.firstChild;e;e=e.nextSibling)if(e.nodeName>"@"||3===e.nodeType||4===e.nodeType)return!1;return!0},parent:function(e){return!i.pseudos.empty(e)},header:function(e){return Q.test(e.nodeName)},input:function(e){return G.test(e.nodeName)},button:function(e){var t=e.nodeName.toLowerCase();return"input"===t&&"button"===e.type||"button"===t},text:function(e){var t;return"input"===e.nodeName.toLowerCase()&&"text"===e.type&&(null==(t=e.getAttribute("type"))||t.toLowerCase()===e.type)},first:pt(function(){return[0]}),last:pt(function(e,t){return[t-1]}),eq:pt(function(e,t,n){return[0>n?n+t:n]}),even:pt(function(e,t){var n=0;for(;t>n;n+=2)e.push(n);return e}),odd:pt(function(e,t){var n=1;for(;t>n;n+=2)e.push(n);return e}),lt:pt(function(e,t,n){var r=0>n?n+t:n;for(;--r>=0;)e.push(r);return e}),gt:pt(function(e,t,n){var r=0>n?n+t:n;for(;t>++r;)e.push(r);return e})}};for(n in{radio:!0,checkbox:!0,file:!0,password:!0,image:!0})i.pseudos[n]=lt(n);for(n in{submit:!0,reset:!0})i.pseudos[n]=ct(n);function ft(e,t){var n,r,o,a,s,u,l,c=E[e+" "];if(c)return t?0:c.slice(0);s=e,u=[],l=i.preFilter;while(s){(!n||(r=$.exec(s)))&&(r&&(s=s.slice(r[0].length)||s),u.push(o=[])),n=!1,(r=I.exec(s))&&(n=r.shift(),o.push({value:n,type:r[0].replace(W," ")}),s=s.slice(n.length));for(a in i.filter)!(r=U[a].exec(s))||l[a]&&!(r=l[a](r))||(n=r.shift(),o.push({value:n,type:a,matches:r}),s=s.slice(n.length));if(!n)break}return t?s.length:s?st.error(e):E(e,u).slice(0)}function dt(e){var t=0,n=e.length,r="";for(;n>t;t++)r+=e[t].value;return r}function ht(e,t,n){var i=t.dir,o=n&&"parentNode"===i,a=C++;return t.first?function(t,n,r){while(t=t[i])if(1===t.nodeType||o)return e(t,n,r)}:function(t,n,s){var u,l,c,p=N+" "+a;if(s){while(t=t[i])if((1===t.nodeType||o)&&e(t,n,s))return!0}else while(t=t[i])if(1===t.nodeType||o)if(c=t[x]||(t[x]={}),(l=c[i])&&l[0]===p){if((u=l[1])===!0||u===r)return u===!0}else if(l=c[i]=[p],l[1]=e(t,n,s)||r,l[1]===!0)return!0}}function gt(e){return e.length>1?function(t,n,r){var i=e.length;while(i--)if(!e[i](t,n,r))return!1;return!0}:e[0]}function mt(e,t,n,r,i){var o,a=[],s=0,u=e.length,l=null!=t;for(;u>s;s++)(o=e[s])&&(!n||n(o,r,i))&&(a.push(o),l&&t.push(s));return a}function yt(e,t,n,r,i,o){return r&&!r[x]&&(r=yt(r)),i&&!i[x]&&(i=yt(i,o)),ot(function(o,a,s,u){var l,c,p,f=[],d=[],h=a.length,g=o||xt(t||"*",s.nodeType?[s]:s,[]),m=!e||!o&&t?g:mt(g,f,e,s,u),y=n?i||(o?e:h||r)?[]:a:m;if(n&&n(m,y,s,u),r){l=mt(y,d),r(l,[],s,u),c=l.length;while(c--)(p=l[c])&&(y[d[c]]=!(m[d[c]]=p))}if(o){if(i||e){if(i){l=[],c=y.length;while(c--)(p=y[c])&&l.push(m[c]=p);i(null,y=[],l,u)}c=y.length;while(c--)(p=y[c])&&(l=i?M.call(o,p):f[c])>-1&&(o[l]=!(a[l]=p))}}else y=mt(y===a?y.splice(h,y.length):y),i?i(null,a,y,u):H.apply(a,y)})}function vt(e){var t,n,r,o=e.length,a=i.relative[e[0].type],s=a||i.relative[" "],u=a?1:0,c=ht(function(e){return e===t},s,!0),p=ht(function(e){return M.call(t,e)>-1},s,!0),f=[function(e,n,r){return!a&&(r||n!==l)||((t=n).nodeType?c(e,n,r):p(e,n,r))}];for(;o>u;u++)if(n=i.relative[e[u].type])f=[ht(gt(f),n)];else{if(n=i.filter[e[u].type].apply(null,e[u].matches),n[x]){for(r=++u;o>r;r++)if(i.relative[e[r].type])break;return yt(u>1&>(f),u>1&&dt(e.slice(0,u-1)).replace(W,"$1"),n,r>u&&vt(e.slice(u,r)),o>r&&vt(e=e.slice(r)),o>r&&dt(e))}f.push(n)}return gt(f)}function bt(e,t){var n=0,o=t.length>0,a=e.length>0,s=function(s,u,c,f,d){var h,g,m,y=[],v=0,b="0",x=s&&[],w=null!=d,T=l,C=s||a&&i.find.TAG("*",d&&u.parentNode||u),k=N+=null==T?1:Math.random()||.1;for(w&&(l=u!==p&&u,r=n);null!=(h=C[b]);b++){if(a&&h){g=0;while(m=e[g++])if(m(h,u,c)){f.push(h);break}w&&(N=k,r=++n)}o&&((h=!m&&h)&&v--,s&&x.push(h))}if(v+=b,o&&b!==v){g=0;while(m=t[g++])m(x,y,u,c);if(s){if(v>0)while(b--)x[b]||y[b]||(y[b]=L.call(f));y=mt(y)}H.apply(f,y),w&&!s&&y.length>0&&v+t.length>1&&st.uniqueSort(f)}return w&&(N=k,l=T),x};return o?ot(s):s}s=st.compile=function(e,t){var n,r=[],i=[],o=S[e+" "];if(!o){t||(t=ft(e)),n=t.length;while(n--)o=vt(t[n]),o[x]?r.push(o):i.push(o);o=S(e,bt(i,r))}return o};function xt(e,t,n){var r=0,i=t.length;for(;i>r;r++)st(e,t[r],n);return n}function wt(e,t,n,r){var o,a,u,l,c,p=ft(e);if(!r&&1===p.length){if(a=p[0]=p[0].slice(0),a.length>2&&"ID"===(u=a[0]).type&&9===t.nodeType&&!d&&i.relative[a[1].type]){if(t=i.find.ID(u.matches[0].replace(et,tt),t)[0],!t)return n;e=e.slice(a.shift().value.length)}o=U.needsContext.test(e)?0:a.length;while(o--){if(u=a[o],i.relative[l=u.type])break;if((c=i.find[l])&&(r=c(u.matches[0].replace(et,tt),V.test(a[0].type)&&t.parentNode||t))){if(a.splice(o,1),e=r.length&&dt(a),!e)return H.apply(n,q.call(r,0)),n;break}}}return s(e,p)(r,t,d,n,V.test(e)),n}i.pseudos.nth=i.pseudos.eq;function Tt(){}i.filters=Tt.prototype=i.pseudos,i.setFilters=new Tt,c(),st.attr=b.attr,b.find=st,b.expr=st.selectors,b.expr[":"]=b.expr.pseudos,b.unique=st.uniqueSort,b.text=st.getText,b.isXMLDoc=st.isXML,b.contains=st.contains}(e);var at=/Until$/,st=/^(?:parents|prev(?:Until|All))/,ut=/^.[^:#\[\.,]*$/,lt=b.expr.match.needsContext,ct={children:!0,contents:!0,next:!0,prev:!0};b.fn.extend({find:function(e){var t,n,r,i=this.length;if("string"!=typeof e)return r=this,this.pushStack(b(e).filter(function(){for(t=0;i>t;t++)if(b.contains(r[t],this))return!0}));for(n=[],t=0;i>t;t++)b.find(e,this[t],n);return n=this.pushStack(i>1?b.unique(n):n),n.selector=(this.selector?this.selector+" ":"")+e,n},has:function(e){var t,n=b(e,this),r=n.length;return this.filter(function(){for(t=0;r>t;t++)if(b.contains(this,n[t]))return!0})},not:function(e){return this.pushStack(ft(this,e,!1))},filter:function(e){return this.pushStack(ft(this,e,!0))},is:function(e){return!!e&&("string"==typeof e?lt.test(e)?b(e,this.context).index(this[0])>=0:b.filter(e,this).length>0:this.filter(e).length>0)},closest:function(e,t){var n,r=0,i=this.length,o=[],a=lt.test(e)||"string"!=typeof e?b(e,t||this.context):0;for(;i>r;r++){n=this[r];while(n&&n.ownerDocument&&n!==t&&11!==n.nodeType){if(a?a.index(n)>-1:b.find.matchesSelector(n,e)){o.push(n);break}n=n.parentNode}}return this.pushStack(o.length>1?b.unique(o):o)},index:function(e){return e?"string"==typeof e?b.inArray(this[0],b(e)):b.inArray(e.jquery?e[0]:e,this):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(e,t){var n="string"==typeof e?b(e,t):b.makeArray(e&&e.nodeType?[e]:e),r=b.merge(this.get(),n);return this.pushStack(b.unique(r))},addBack:function(e){return this.add(null==e?this.prevObject:this.prevObject.filter(e))}}),b.fn.andSelf=b.fn.addBack;function pt(e,t){do e=e[t];while(e&&1!==e.nodeType);return e}b.each({parent:function(e){var t=e.parentNode;return t&&11!==t.nodeType?t:null},parents:function(e){return b.dir(e,"parentNode")},parentsUntil:function(e,t,n){return b.dir(e,"parentNode",n)},next:function(e){return pt(e,"nextSibling")},prev:function(e){return pt(e,"previousSibling")},nextAll:function(e){return b.dir(e,"nextSibling")},prevAll:function(e){return b.dir(e,"previousSibling")},nextUntil:function(e,t,n){return b.dir(e,"nextSibling",n)},prevUntil:function(e,t,n){return b.dir(e,"previousSibling",n)},siblings:function(e){return b.sibling((e.parentNode||{}).firstChild,e)},children:function(e){return b.sibling(e.firstChild)},contents:function(e){return b.nodeName(e,"iframe")?e.contentDocument||e.contentWindow.document:b.merge([],e.childNodes)}},function(e,t){b.fn[e]=function(n,r){var i=b.map(this,t,n);return at.test(e)||(r=n),r&&"string"==typeof r&&(i=b.filter(r,i)),i=this.length>1&&!ct[e]?b.unique(i):i,this.length>1&&st.test(e)&&(i=i.reverse()),this.pushStack(i)}}),b.extend({filter:function(e,t,n){return n&&(e=":not("+e+")"),1===t.length?b.find.matchesSelector(t[0],e)?[t[0]]:[]:b.find.matches(e,t)},dir:function(e,n,r){var i=[],o=e[n];while(o&&9!==o.nodeType&&(r===t||1!==o.nodeType||!b(o).is(r)))1===o.nodeType&&i.push(o),o=o[n];return i},sibling:function(e,t){var n=[];for(;e;e=e.nextSibling)1===e.nodeType&&e!==t&&n.push(e);return n}});function ft(e,t,n){if(t=t||0,b.isFunction(t))return b.grep(e,function(e,r){var i=!!t.call(e,r,e);return i===n});if(t.nodeType)return b.grep(e,function(e){return e===t===n});if("string"==typeof t){var r=b.grep(e,function(e){return 1===e.nodeType});if(ut.test(t))return b.filter(t,r,!n);t=b.filter(t,r)}return b.grep(e,function(e){return b.inArray(e,t)>=0===n})}function dt(e){var t=ht.split("|"),n=e.createDocumentFragment();if(n.createElement)while(t.length)n.createElement(t.pop());return n}var ht="abbr|article|aside|audio|bdi|canvas|data|datalist|details|figcaption|figure|footer|header|hgroup|mark|meter|nav|output|progress|section|summary|time|video",gt=/ jQuery\d+="(?:null|\d+)"/g,mt=RegExp("<(?:"+ht+")[\\s/>]","i"),yt=/^\s+/,vt=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/gi,bt=/<([\w:]+)/,xt=/<tbody/i,wt=/<|&#?\w+;/,Tt=/<(?:script|style|link)/i,Nt=/^(?:checkbox|radio)$/i,Ct=/checked\s*(?:[^=]|=\s*.checked.)/i,kt=/^$|\/(?:java|ecma)script/i,Et=/^true\/(.*)/,St=/^\s*<!(?:\[CDATA\[|--)|(?:\]\]|--)>\s*$/g,At={option:[1,"<select multiple='multiple'>","</select>"],legend:[1,"<fieldset>","</fieldset>"],area:[1,"<map>","</map>"],param:[1,"<object>","</object>"],thead:[1,"<table>","</table>"],tr:[2,"<table><tbody>","</tbody></table>"],col:[2,"<table><tbody></tbody><colgroup>","</colgroup></table>"],td:[3,"<table><tbody><tr>","</tr></tbody></table>"],_default:b.support.htmlSerialize?[0,"",""]:[1,"X<div>","</div>"]},jt=dt(o),Dt=jt.appendChild(o.createElement("div"));At.optgroup=At.option,At.tbody=At.tfoot=At.colgroup=At.caption=At.thead,At.th=At.td,b.fn.extend({text:function(e){return b.access(this,function(e){return e===t?b.text(this):this.empty().append((this[0]&&this[0].ownerDocument||o).createTextNode(e))},null,e,arguments.length)},wrapAll:function(e){if(b.isFunction(e))return this.each(function(t){b(this).wrapAll(e.call(this,t))});if(this[0]){var t=b(e,this[0].ownerDocument).eq(0).clone(!0);this[0].parentNode&&t.insertBefore(this[0]),t.map(function(){var e=this;while(e.firstChild&&1===e.firstChild.nodeType)e=e.firstChild;return e}).append(this)}return this},wrapInner:function(e){return b.isFunction(e)?this.each(function(t){b(this).wrapInner(e.call(this,t))}):this.each(function(){var t=b(this),n=t.contents();n.length?n.wrapAll(e):t.append(e)})},wrap:function(e){var t=b.isFunction(e);return this.each(function(n){b(this).wrapAll(t?e.call(this,n):e)})},unwrap:function(){return this.parent().each(function(){b.nodeName(this,"body")||b(this).replaceWith(this.childNodes)}).end()},append:function(){return this.domManip(arguments,!0,function(e){(1===this.nodeType||11===this.nodeType||9===this.nodeType)&&this.appendChild(e)})},prepend:function(){return this.domManip(arguments,!0,function(e){(1===this.nodeType||11===this.nodeType||9===this.nodeType)&&this.insertBefore(e,this.firstChild)})},before:function(){return this.domManip(arguments,!1,function(e){this.parentNode&&this.parentNode.insertBefore(e,this)})},after:function(){return this.domManip(arguments,!1,function(e){this.parentNode&&this.parentNode.insertBefore(e,this.nextSibling)})},remove:function(e,t){var n,r=0;for(;null!=(n=this[r]);r++)(!e||b.filter(e,[n]).length>0)&&(t||1!==n.nodeType||b.cleanData(Ot(n)),n.parentNode&&(t&&b.contains(n.ownerDocument,n)&&Mt(Ot(n,"script")),n.parentNode.removeChild(n)));return this},empty:function(){var e,t=0;for(;null!=(e=this[t]);t++){1===e.nodeType&&b.cleanData(Ot(e,!1));while(e.firstChild)e.removeChild(e.firstChild);e.options&&b.nodeName(e,"select")&&(e.options.length=0)}return this},clone:function(e,t){return e=null==e?!1:e,t=null==t?e:t,this.map(function(){return b.clone(this,e,t)})},html:function(e){return b.access(this,function(e){var n=this[0]||{},r=0,i=this.length;if(e===t)return 1===n.nodeType?n.innerHTML.replace(gt,""):t;if(!("string"!=typeof e||Tt.test(e)||!b.support.htmlSerialize&&mt.test(e)||!b.support.leadingWhitespace&&yt.test(e)||At[(bt.exec(e)||["",""])[1].toLowerCase()])){e=e.replace(vt,"<$1></$2>");try{for(;i>r;r++)n=this[r]||{},1===n.nodeType&&(b.cleanData(Ot(n,!1)),n.innerHTML=e);n=0}catch(o){}}n&&this.empty().append(e)},null,e,arguments.length)},replaceWith:function(e){var t=b.isFunction(e);return t||"string"==typeof e||(e=b(e).not(this).detach()),this.domManip([e],!0,function(e){var t=this.nextSibling,n=this.parentNode;n&&(b(this).remove(),n.insertBefore(e,t))})},detach:function(e){return this.remove(e,!0)},domManip:function(e,n,r){e=f.apply([],e);var i,o,a,s,u,l,c=0,p=this.length,d=this,h=p-1,g=e[0],m=b.isFunction(g);if(m||!(1>=p||"string"!=typeof g||b.support.checkClone)&&Ct.test(g))return this.each(function(i){var o=d.eq(i);m&&(e[0]=g.call(this,i,n?o.html():t)),o.domManip(e,n,r)});if(p&&(l=b.buildFragment(e,this[0].ownerDocument,!1,this),i=l.firstChild,1===l.childNodes.length&&(l=i),i)){for(n=n&&b.nodeName(i,"tr"),s=b.map(Ot(l,"script"),Ht),a=s.length;p>c;c++)o=l,c!==h&&(o=b.clone(o,!0,!0),a&&b.merge(s,Ot(o,"script"))),r.call(n&&b.nodeName(this[c],"table")?Lt(this[c],"tbody"):this[c],o,c);if(a)for(u=s[s.length-1].ownerDocument,b.map(s,qt),c=0;a>c;c++)o=s[c],kt.test(o.type||"")&&!b._data(o,"globalEval")&&b.contains(u,o)&&(o.src?b.ajax({url:o.src,type:"GET",dataType:"script",async:!1,global:!1,"throws":!0}):b.globalEval((o.text||o.textContent||o.innerHTML||"").replace(St,"")));l=i=null}return this}});function Lt(e,t){return e.getElementsByTagName(t)[0]||e.appendChild(e.ownerDocument.createElement(t))}function Ht(e){var t=e.getAttributeNode("type");return e.type=(t&&t.specified)+"/"+e.type,e}function qt(e){var t=Et.exec(e.type);return t?e.type=t[1]:e.removeAttribute("type"),e}function Mt(e,t){var n,r=0;for(;null!=(n=e[r]);r++)b._data(n,"globalEval",!t||b._data(t[r],"globalEval"))}function _t(e,t){if(1===t.nodeType&&b.hasData(e)){var n,r,i,o=b._data(e),a=b._data(t,o),s=o.events;if(s){delete a.handle,a.events={};for(n in s)for(r=0,i=s[n].length;i>r;r++)b.event.add(t,n,s[n][r])}a.data&&(a.data=b.extend({},a.data))}}function Ft(e,t){var n,r,i;if(1===t.nodeType){if(n=t.nodeName.toLowerCase(),!b.support.noCloneEvent&&t[b.expando]){i=b._data(t);for(r in i.events)b.removeEvent(t,r,i.handle);t.removeAttribute(b.expando)}"script"===n&&t.text!==e.text?(Ht(t).text=e.text,qt(t)):"object"===n?(t.parentNode&&(t.outerHTML=e.outerHTML),b.support.html5Clone&&e.innerHTML&&!b.trim(t.innerHTML)&&(t.innerHTML=e.innerHTML)):"input"===n&&Nt.test(e.type)?(t.defaultChecked=t.checked=e.checked,t.value!==e.value&&(t.value=e.value)):"option"===n?t.defaultSelected=t.selected=e.defaultSelected:("input"===n||"textarea"===n)&&(t.defaultValue=e.defaultValue)}}b.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(e,t){b.fn[e]=function(e){var n,r=0,i=[],o=b(e),a=o.length-1;for(;a>=r;r++)n=r===a?this:this.clone(!0),b(o[r])[t](n),d.apply(i,n.get());return this.pushStack(i)}});function Ot(e,n){var r,o,a=0,s=typeof e.getElementsByTagName!==i?e.getElementsByTagName(n||"*"):typeof e.querySelectorAll!==i?e.querySelectorAll(n||"*"):t;if(!s)for(s=[],r=e.childNodes||e;null!=(o=r[a]);a++)!n||b.nodeName(o,n)?s.push(o):b.merge(s,Ot(o,n));return n===t||n&&b.nodeName(e,n)?b.merge([e],s):s}function Bt(e){Nt.test(e.type)&&(e.defaultChecked=e.checked)}b.extend({clone:function(e,t,n){var r,i,o,a,s,u=b.contains(e.ownerDocument,e);if(b.support.html5Clone||b.isXMLDoc(e)||!mt.test("<"+e.nodeName+">")?o=e.cloneNode(!0):(Dt.innerHTML=e.outerHTML,Dt.removeChild(o=Dt.firstChild)),!(b.support.noCloneEvent&&b.support.noCloneChecked||1!==e.nodeType&&11!==e.nodeType||b.isXMLDoc(e)))for(r=Ot(o),s=Ot(e),a=0;null!=(i=s[a]);++a)r[a]&&Ft(i,r[a]);if(t)if(n)for(s=s||Ot(e),r=r||Ot(o),a=0;null!=(i=s[a]);a++)_t(i,r[a]);else _t(e,o);return r=Ot(o,"script"),r.length>0&&Mt(r,!u&&Ot(e,"script")),r=s=i=null,o},buildFragment:function(e,t,n,r){var i,o,a,s,u,l,c,p=e.length,f=dt(t),d=[],h=0;for(;p>h;h++)if(o=e[h],o||0===o)if("object"===b.type(o))b.merge(d,o.nodeType?[o]:o);else if(wt.test(o)){s=s||f.appendChild(t.createElement("div")),u=(bt.exec(o)||["",""])[1].toLowerCase(),c=At[u]||At._default,s.innerHTML=c[1]+o.replace(vt,"<$1></$2>")+c[2],i=c[0];while(i--)s=s.lastChild;if(!b.support.leadingWhitespace&&yt.test(o)&&d.push(t.createTextNode(yt.exec(o)[0])),!b.support.tbody){o="table"!==u||xt.test(o)?"<table>"!==c[1]||xt.test(o)?0:s:s.firstChild,i=o&&o.childNodes.length;while(i--)b.nodeName(l=o.childNodes[i],"tbody")&&!l.childNodes.length&&o.removeChild(l)
-}b.merge(d,s.childNodes),s.textContent="";while(s.firstChild)s.removeChild(s.firstChild);s=f.lastChild}else d.push(t.createTextNode(o));s&&f.removeChild(s),b.support.appendChecked||b.grep(Ot(d,"input"),Bt),h=0;while(o=d[h++])if((!r||-1===b.inArray(o,r))&&(a=b.contains(o.ownerDocument,o),s=Ot(f.appendChild(o),"script"),a&&Mt(s),n)){i=0;while(o=s[i++])kt.test(o.type||"")&&n.push(o)}return s=null,f},cleanData:function(e,t){var n,r,o,a,s=0,u=b.expando,l=b.cache,p=b.support.deleteExpando,f=b.event.special;for(;null!=(n=e[s]);s++)if((t||b.acceptData(n))&&(o=n[u],a=o&&l[o])){if(a.events)for(r in a.events)f[r]?b.event.remove(n,r):b.removeEvent(n,r,a.handle);l[o]&&(delete l[o],p?delete n[u]:typeof n.removeAttribute!==i?n.removeAttribute(u):n[u]=null,c.push(o))}}});var Pt,Rt,Wt,$t=/alpha\([^)]*\)/i,It=/opacity\s*=\s*([^)]*)/,zt=/^(top|right|bottom|left)$/,Xt=/^(none|table(?!-c[ea]).+)/,Ut=/^margin/,Vt=RegExp("^("+x+")(.*)$","i"),Yt=RegExp("^("+x+")(?!px)[a-z%]+$","i"),Jt=RegExp("^([+-])=("+x+")","i"),Gt={BODY:"block"},Qt={position:"absolute",visibility:"hidden",display:"block"},Kt={letterSpacing:0,fontWeight:400},Zt=["Top","Right","Bottom","Left"],en=["Webkit","O","Moz","ms"];function tn(e,t){if(t in e)return t;var n=t.charAt(0).toUpperCase()+t.slice(1),r=t,i=en.length;while(i--)if(t=en[i]+n,t in e)return t;return r}function nn(e,t){return e=t||e,"none"===b.css(e,"display")||!b.contains(e.ownerDocument,e)}function rn(e,t){var n,r,i,o=[],a=0,s=e.length;for(;s>a;a++)r=e[a],r.style&&(o[a]=b._data(r,"olddisplay"),n=r.style.display,t?(o[a]||"none"!==n||(r.style.display=""),""===r.style.display&&nn(r)&&(o[a]=b._data(r,"olddisplay",un(r.nodeName)))):o[a]||(i=nn(r),(n&&"none"!==n||!i)&&b._data(r,"olddisplay",i?n:b.css(r,"display"))));for(a=0;s>a;a++)r=e[a],r.style&&(t&&"none"!==r.style.display&&""!==r.style.display||(r.style.display=t?o[a]||"":"none"));return e}b.fn.extend({css:function(e,n){return b.access(this,function(e,n,r){var i,o,a={},s=0;if(b.isArray(n)){for(o=Rt(e),i=n.length;i>s;s++)a[n[s]]=b.css(e,n[s],!1,o);return a}return r!==t?b.style(e,n,r):b.css(e,n)},e,n,arguments.length>1)},show:function(){return rn(this,!0)},hide:function(){return rn(this)},toggle:function(e){var t="boolean"==typeof e;return this.each(function(){(t?e:nn(this))?b(this).show():b(this).hide()})}}),b.extend({cssHooks:{opacity:{get:function(e,t){if(t){var n=Wt(e,"opacity");return""===n?"1":n}}}},cssNumber:{columnCount:!0,fillOpacity:!0,fontWeight:!0,lineHeight:!0,opacity:!0,orphans:!0,widows:!0,zIndex:!0,zoom:!0},cssProps:{"float":b.support.cssFloat?"cssFloat":"styleFloat"},style:function(e,n,r,i){if(e&&3!==e.nodeType&&8!==e.nodeType&&e.style){var o,a,s,u=b.camelCase(n),l=e.style;if(n=b.cssProps[u]||(b.cssProps[u]=tn(l,u)),s=b.cssHooks[n]||b.cssHooks[u],r===t)return s&&"get"in s&&(o=s.get(e,!1,i))!==t?o:l[n];if(a=typeof r,"string"===a&&(o=Jt.exec(r))&&(r=(o[1]+1)*o[2]+parseFloat(b.css(e,n)),a="number"),!(null==r||"number"===a&&isNaN(r)||("number"!==a||b.cssNumber[u]||(r+="px"),b.support.clearCloneStyle||""!==r||0!==n.indexOf("background")||(l[n]="inherit"),s&&"set"in s&&(r=s.set(e,r,i))===t)))try{l[n]=r}catch(c){}}},css:function(e,n,r,i){var o,a,s,u=b.camelCase(n);return n=b.cssProps[u]||(b.cssProps[u]=tn(e.style,u)),s=b.cssHooks[n]||b.cssHooks[u],s&&"get"in s&&(a=s.get(e,!0,r)),a===t&&(a=Wt(e,n,i)),"normal"===a&&n in Kt&&(a=Kt[n]),""===r||r?(o=parseFloat(a),r===!0||b.isNumeric(o)?o||0:a):a},swap:function(e,t,n,r){var i,o,a={};for(o in t)a[o]=e.style[o],e.style[o]=t[o];i=n.apply(e,r||[]);for(o in t)e.style[o]=a[o];return i}}),e.getComputedStyle?(Rt=function(t){return e.getComputedStyle(t,null)},Wt=function(e,n,r){var i,o,a,s=r||Rt(e),u=s?s.getPropertyValue(n)||s[n]:t,l=e.style;return s&&(""!==u||b.contains(e.ownerDocument,e)||(u=b.style(e,n)),Yt.test(u)&&Ut.test(n)&&(i=l.width,o=l.minWidth,a=l.maxWidth,l.minWidth=l.maxWidth=l.width=u,u=s.width,l.width=i,l.minWidth=o,l.maxWidth=a)),u}):o.documentElement.currentStyle&&(Rt=function(e){return e.currentStyle},Wt=function(e,n,r){var i,o,a,s=r||Rt(e),u=s?s[n]:t,l=e.style;return null==u&&l&&l[n]&&(u=l[n]),Yt.test(u)&&!zt.test(n)&&(i=l.left,o=e.runtimeStyle,a=o&&o.left,a&&(o.left=e.currentStyle.left),l.left="fontSize"===n?"1em":u,u=l.pixelLeft+"px",l.left=i,a&&(o.left=a)),""===u?"auto":u});function on(e,t,n){var r=Vt.exec(t);return r?Math.max(0,r[1]-(n||0))+(r[2]||"px"):t}function an(e,t,n,r,i){var o=n===(r?"border":"content")?4:"width"===t?1:0,a=0;for(;4>o;o+=2)"margin"===n&&(a+=b.css(e,n+Zt[o],!0,i)),r?("content"===n&&(a-=b.css(e,"padding"+Zt[o],!0,i)),"margin"!==n&&(a-=b.css(e,"border"+Zt[o]+"Width",!0,i))):(a+=b.css(e,"padding"+Zt[o],!0,i),"padding"!==n&&(a+=b.css(e,"border"+Zt[o]+"Width",!0,i)));return a}function sn(e,t,n){var r=!0,i="width"===t?e.offsetWidth:e.offsetHeight,o=Rt(e),a=b.support.boxSizing&&"border-box"===b.css(e,"boxSizing",!1,o);if(0>=i||null==i){if(i=Wt(e,t,o),(0>i||null==i)&&(i=e.style[t]),Yt.test(i))return i;r=a&&(b.support.boxSizingReliable||i===e.style[t]),i=parseFloat(i)||0}return i+an(e,t,n||(a?"border":"content"),r,o)+"px"}function un(e){var t=o,n=Gt[e];return n||(n=ln(e,t),"none"!==n&&n||(Pt=(Pt||b("<iframe frameborder='0' width='0' height='0'/>").css("cssText","display:block !important")).appendTo(t.documentElement),t=(Pt[0].contentWindow||Pt[0].contentDocument).document,t.write("<!doctype html><html><body>"),t.close(),n=ln(e,t),Pt.detach()),Gt[e]=n),n}function ln(e,t){var n=b(t.createElement(e)).appendTo(t.body),r=b.css(n[0],"display");return n.remove(),r}b.each(["height","width"],function(e,n){b.cssHooks[n]={get:function(e,r,i){return r?0===e.offsetWidth&&Xt.test(b.css(e,"display"))?b.swap(e,Qt,function(){return sn(e,n,i)}):sn(e,n,i):t},set:function(e,t,r){var i=r&&Rt(e);return on(e,t,r?an(e,n,r,b.support.boxSizing&&"border-box"===b.css(e,"boxSizing",!1,i),i):0)}}}),b.support.opacity||(b.cssHooks.opacity={get:function(e,t){return It.test((t&&e.currentStyle?e.currentStyle.filter:e.style.filter)||"")?.01*parseFloat(RegExp.$1)+"":t?"1":""},set:function(e,t){var n=e.style,r=e.currentStyle,i=b.isNumeric(t)?"alpha(opacity="+100*t+")":"",o=r&&r.filter||n.filter||"";n.zoom=1,(t>=1||""===t)&&""===b.trim(o.replace($t,""))&&n.removeAttribute&&(n.removeAttribute("filter"),""===t||r&&!r.filter)||(n.filter=$t.test(o)?o.replace($t,i):o+" "+i)}}),b(function(){b.support.reliableMarginRight||(b.cssHooks.marginRight={get:function(e,n){return n?b.swap(e,{display:"inline-block"},Wt,[e,"marginRight"]):t}}),!b.support.pixelPosition&&b.fn.position&&b.each(["top","left"],function(e,n){b.cssHooks[n]={get:function(e,r){return r?(r=Wt(e,n),Yt.test(r)?b(e).position()[n]+"px":r):t}}})}),b.expr&&b.expr.filters&&(b.expr.filters.hidden=function(e){return 0>=e.offsetWidth&&0>=e.offsetHeight||!b.support.reliableHiddenOffsets&&"none"===(e.style&&e.style.display||b.css(e,"display"))},b.expr.filters.visible=function(e){return!b.expr.filters.hidden(e)}),b.each({margin:"",padding:"",border:"Width"},function(e,t){b.cssHooks[e+t]={expand:function(n){var r=0,i={},o="string"==typeof n?n.split(" "):[n];for(;4>r;r++)i[e+Zt[r]+t]=o[r]||o[r-2]||o[0];return i}},Ut.test(e)||(b.cssHooks[e+t].set=on)});var cn=/%20/g,pn=/\[\]$/,fn=/\r?\n/g,dn=/^(?:submit|button|image|reset|file)$/i,hn=/^(?:input|select|textarea|keygen)/i;b.fn.extend({serialize:function(){return b.param(this.serializeArray())},serializeArray:function(){return this.map(function(){var e=b.prop(this,"elements");return e?b.makeArray(e):this}).filter(function(){var e=this.type;return this.name&&!b(this).is(":disabled")&&hn.test(this.nodeName)&&!dn.test(e)&&(this.checked||!Nt.test(e))}).map(function(e,t){var n=b(this).val();return null==n?null:b.isArray(n)?b.map(n,function(e){return{name:t.name,value:e.replace(fn,"\r\n")}}):{name:t.name,value:n.replace(fn,"\r\n")}}).get()}}),b.param=function(e,n){var r,i=[],o=function(e,t){t=b.isFunction(t)?t():null==t?"":t,i[i.length]=encodeURIComponent(e)+"="+encodeURIComponent(t)};if(n===t&&(n=b.ajaxSettings&&b.ajaxSettings.traditional),b.isArray(e)||e.jquery&&!b.isPlainObject(e))b.each(e,function(){o(this.name,this.value)});else for(r in e)gn(r,e[r],n,o);return i.join("&").replace(cn,"+")};function gn(e,t,n,r){var i;if(b.isArray(t))b.each(t,function(t,i){n||pn.test(e)?r(e,i):gn(e+"["+("object"==typeof i?t:"")+"]",i,n,r)});else if(n||"object"!==b.type(t))r(e,t);else for(i in t)gn(e+"["+i+"]",t[i],n,r)}b.each("blur focus focusin focusout load resize scroll unload click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup error contextmenu".split(" "),function(e,t){b.fn[t]=function(e,n){return arguments.length>0?this.on(t,null,e,n):this.trigger(t)}}),b.fn.hover=function(e,t){return this.mouseenter(e).mouseleave(t||e)};var mn,yn,vn=b.now(),bn=/\?/,xn=/#.*$/,wn=/([?&])_=[^&]*/,Tn=/^(.*?):[ \t]*([^\r\n]*)\r?$/gm,Nn=/^(?:about|app|app-storage|.+-extension|file|res|widget):$/,Cn=/^(?:GET|HEAD)$/,kn=/^\/\//,En=/^([\w.+-]+:)(?:\/\/([^\/?#:]*)(?::(\d+)|)|)/,Sn=b.fn.load,An={},jn={},Dn="*/".concat("*");try{yn=a.href}catch(Ln){yn=o.createElement("a"),yn.href="",yn=yn.href}mn=En.exec(yn.toLowerCase())||[];function Hn(e){return function(t,n){"string"!=typeof t&&(n=t,t="*");var r,i=0,o=t.toLowerCase().match(w)||[];if(b.isFunction(n))while(r=o[i++])"+"===r[0]?(r=r.slice(1)||"*",(e[r]=e[r]||[]).unshift(n)):(e[r]=e[r]||[]).push(n)}}function qn(e,n,r,i){var o={},a=e===jn;function s(u){var l;return o[u]=!0,b.each(e[u]||[],function(e,u){var c=u(n,r,i);return"string"!=typeof c||a||o[c]?a?!(l=c):t:(n.dataTypes.unshift(c),s(c),!1)}),l}return s(n.dataTypes[0])||!o["*"]&&s("*")}function Mn(e,n){var r,i,o=b.ajaxSettings.flatOptions||{};for(i in n)n[i]!==t&&((o[i]?e:r||(r={}))[i]=n[i]);return r&&b.extend(!0,e,r),e}b.fn.load=function(e,n,r){if("string"!=typeof e&&Sn)return Sn.apply(this,arguments);var i,o,a,s=this,u=e.indexOf(" ");return u>=0&&(i=e.slice(u,e.length),e=e.slice(0,u)),b.isFunction(n)?(r=n,n=t):n&&"object"==typeof n&&(a="POST"),s.length>0&&b.ajax({url:e,type:a,dataType:"html",data:n}).done(function(e){o=arguments,s.html(i?b("<div>").append(b.parseHTML(e)).find(i):e)}).complete(r&&function(e,t){s.each(r,o||[e.responseText,t,e])}),this},b.each(["ajaxStart","ajaxStop","ajaxComplete","ajaxError","ajaxSuccess","ajaxSend"],function(e,t){b.fn[t]=function(e){return this.on(t,e)}}),b.each(["get","post"],function(e,n){b[n]=function(e,r,i,o){return b.isFunction(r)&&(o=o||i,i=r,r=t),b.ajax({url:e,type:n,dataType:o,data:r,success:i})}}),b.extend({active:0,lastModified:{},etag:{},ajaxSettings:{url:yn,type:"GET",isLocal:Nn.test(mn[1]),global:!0,processData:!0,async:!0,contentType:"application/x-www-form-urlencoded; charset=UTF-8",accepts:{"*":Dn,text:"text/plain",html:"text/html",xml:"application/xml, text/xml",json:"application/json, text/javascript"},contents:{xml:/xml/,html:/html/,json:/json/},responseFields:{xml:"responseXML",text:"responseText"},converters:{"* text":e.String,"text html":!0,"text json":b.parseJSON,"text xml":b.parseXML},flatOptions:{url:!0,context:!0}},ajaxSetup:function(e,t){return t?Mn(Mn(e,b.ajaxSettings),t):Mn(b.ajaxSettings,e)},ajaxPrefilter:Hn(An),ajaxTransport:Hn(jn),ajax:function(e,n){"object"==typeof e&&(n=e,e=t),n=n||{};var r,i,o,a,s,u,l,c,p=b.ajaxSetup({},n),f=p.context||p,d=p.context&&(f.nodeType||f.jquery)?b(f):b.event,h=b.Deferred(),g=b.Callbacks("once memory"),m=p.statusCode||{},y={},v={},x=0,T="canceled",N={readyState:0,getResponseHeader:function(e){var t;if(2===x){if(!c){c={};while(t=Tn.exec(a))c[t[1].toLowerCase()]=t[2]}t=c[e.toLowerCase()]}return null==t?null:t},getAllResponseHeaders:function(){return 2===x?a:null},setRequestHeader:function(e,t){var n=e.toLowerCase();return x||(e=v[n]=v[n]||e,y[e]=t),this},overrideMimeType:function(e){return x||(p.mimeType=e),this},statusCode:function(e){var t;if(e)if(2>x)for(t in e)m[t]=[m[t],e[t]];else N.always(e[N.status]);return this},abort:function(e){var t=e||T;return l&&l.abort(t),k(0,t),this}};if(h.promise(N).complete=g.add,N.success=N.done,N.error=N.fail,p.url=((e||p.url||yn)+"").replace(xn,"").replace(kn,mn[1]+"//"),p.type=n.method||n.type||p.method||p.type,p.dataTypes=b.trim(p.dataType||"*").toLowerCase().match(w)||[""],null==p.crossDomain&&(r=En.exec(p.url.toLowerCase()),p.crossDomain=!(!r||r[1]===mn[1]&&r[2]===mn[2]&&(r[3]||("http:"===r[1]?80:443))==(mn[3]||("http:"===mn[1]?80:443)))),p.data&&p.processData&&"string"!=typeof p.data&&(p.data=b.param(p.data,p.traditional)),qn(An,p,n,N),2===x)return N;u=p.global,u&&0===b.active++&&b.event.trigger("ajaxStart"),p.type=p.type.toUpperCase(),p.hasContent=!Cn.test(p.type),o=p.url,p.hasContent||(p.data&&(o=p.url+=(bn.test(o)?"&":"?")+p.data,delete p.data),p.cache===!1&&(p.url=wn.test(o)?o.replace(wn,"$1_="+vn++):o+(bn.test(o)?"&":"?")+"_="+vn++)),p.ifModified&&(b.lastModified[o]&&N.setRequestHeader("If-Modified-Since",b.lastModified[o]),b.etag[o]&&N.setRequestHeader("If-None-Match",b.etag[o])),(p.data&&p.hasContent&&p.contentType!==!1||n.contentType)&&N.setRequestHeader("Content-Type",p.contentType),N.setRequestHeader("Accept",p.dataTypes[0]&&p.accepts[p.dataTypes[0]]?p.accepts[p.dataTypes[0]]+("*"!==p.dataTypes[0]?", "+Dn+"; q=0.01":""):p.accepts["*"]);for(i in p.headers)N.setRequestHeader(i,p.headers[i]);if(p.beforeSend&&(p.beforeSend.call(f,N,p)===!1||2===x))return N.abort();T="abort";for(i in{success:1,error:1,complete:1})N[i](p[i]);if(l=qn(jn,p,n,N)){N.readyState=1,u&&d.trigger("ajaxSend",[N,p]),p.async&&p.timeout>0&&(s=setTimeout(function(){N.abort("timeout")},p.timeout));try{x=1,l.send(y,k)}catch(C){if(!(2>x))throw C;k(-1,C)}}else k(-1,"No Transport");function k(e,n,r,i){var c,y,v,w,T,C=n;2!==x&&(x=2,s&&clearTimeout(s),l=t,a=i||"",N.readyState=e>0?4:0,r&&(w=_n(p,N,r)),e>=200&&300>e||304===e?(p.ifModified&&(T=N.getResponseHeader("Last-Modified"),T&&(b.lastModified[o]=T),T=N.getResponseHeader("etag"),T&&(b.etag[o]=T)),204===e?(c=!0,C="nocontent"):304===e?(c=!0,C="notmodified"):(c=Fn(p,w),C=c.state,y=c.data,v=c.error,c=!v)):(v=C,(e||!C)&&(C="error",0>e&&(e=0))),N.status=e,N.statusText=(n||C)+"",c?h.resolveWith(f,[y,C,N]):h.rejectWith(f,[N,C,v]),N.statusCode(m),m=t,u&&d.trigger(c?"ajaxSuccess":"ajaxError",[N,p,c?y:v]),g.fireWith(f,[N,C]),u&&(d.trigger("ajaxComplete",[N,p]),--b.active||b.event.trigger("ajaxStop")))}return N},getScript:function(e,n){return b.get(e,t,n,"script")},getJSON:function(e,t,n){return b.get(e,t,n,"json")}});function _n(e,n,r){var i,o,a,s,u=e.contents,l=e.dataTypes,c=e.responseFields;for(s in c)s in r&&(n[c[s]]=r[s]);while("*"===l[0])l.shift(),o===t&&(o=e.mimeType||n.getResponseHeader("Content-Type"));if(o)for(s in u)if(u[s]&&u[s].test(o)){l.unshift(s);break}if(l[0]in r)a=l[0];else{for(s in r){if(!l[0]||e.converters[s+" "+l[0]]){a=s;break}i||(i=s)}a=a||i}return a?(a!==l[0]&&l.unshift(a),r[a]):t}function Fn(e,t){var n,r,i,o,a={},s=0,u=e.dataTypes.slice(),l=u[0];if(e.dataFilter&&(t=e.dataFilter(t,e.dataType)),u[1])for(i in e.converters)a[i.toLowerCase()]=e.converters[i];for(;r=u[++s];)if("*"!==r){if("*"!==l&&l!==r){if(i=a[l+" "+r]||a["* "+r],!i)for(n in a)if(o=n.split(" "),o[1]===r&&(i=a[l+" "+o[0]]||a["* "+o[0]])){i===!0?i=a[n]:a[n]!==!0&&(r=o[0],u.splice(s--,0,r));break}if(i!==!0)if(i&&e["throws"])t=i(t);else try{t=i(t)}catch(c){return{state:"parsererror",error:i?c:"No conversion from "+l+" to "+r}}}l=r}return{state:"success",data:t}}b.ajaxSetup({accepts:{script:"text/javascript, application/javascript, application/ecmascript, application/x-ecmascript"},contents:{script:/(?:java|ecma)script/},converters:{"text script":function(e){return b.globalEval(e),e}}}),b.ajaxPrefilter("script",function(e){e.cache===t&&(e.cache=!1),e.crossDomain&&(e.type="GET",e.global=!1)}),b.ajaxTransport("script",function(e){if(e.crossDomain){var n,r=o.head||b("head")[0]||o.documentElement;return{send:function(t,i){n=o.createElement("script"),n.async=!0,e.scriptCharset&&(n.charset=e.scriptCharset),n.src=e.url,n.onload=n.onreadystatechange=function(e,t){(t||!n.readyState||/loaded|complete/.test(n.readyState))&&(n.onload=n.onreadystatechange=null,n.parentNode&&n.parentNode.removeChild(n),n=null,t||i(200,"success"))},r.insertBefore(n,r.firstChild)},abort:function(){n&&n.onload(t,!0)}}}});var On=[],Bn=/(=)\?(?=&|$)|\?\?/;b.ajaxSetup({jsonp:"callback",jsonpCallback:function(){var e=On.pop()||b.expando+"_"+vn++;return this[e]=!0,e}}),b.ajaxPrefilter("json jsonp",function(n,r,i){var o,a,s,u=n.jsonp!==!1&&(Bn.test(n.url)?"url":"string"==typeof n.data&&!(n.contentType||"").indexOf("application/x-www-form-urlencoded")&&Bn.test(n.data)&&"data");return u||"jsonp"===n.dataTypes[0]?(o=n.jsonpCallback=b.isFunction(n.jsonpCallback)?n.jsonpCallback():n.jsonpCallback,u?n[u]=n[u].replace(Bn,"$1"+o):n.jsonp!==!1&&(n.url+=(bn.test(n.url)?"&":"?")+n.jsonp+"="+o),n.converters["script json"]=function(){return s||b.error(o+" was not called"),s[0]},n.dataTypes[0]="json",a=e[o],e[o]=function(){s=arguments},i.always(function(){e[o]=a,n[o]&&(n.jsonpCallback=r.jsonpCallback,On.push(o)),s&&b.isFunction(a)&&a(s[0]),s=a=t}),"script"):t});var Pn,Rn,Wn=0,$n=e.ActiveXObject&&function(){var e;for(e in Pn)Pn[e](t,!0)};function In(){try{return new e.XMLHttpRequest}catch(t){}}function zn(){try{return new e.ActiveXObject("Microsoft.XMLHTTP")}catch(t){}}b.ajaxSettings.xhr=e.ActiveXObject?function(){return!this.isLocal&&In()||zn()}:In,Rn=b.ajaxSettings.xhr(),b.support.cors=!!Rn&&"withCredentials"in Rn,Rn=b.support.ajax=!!Rn,Rn&&b.ajaxTransport(function(n){if(!n.crossDomain||b.support.cors){var r;return{send:function(i,o){var a,s,u=n.xhr();if(n.username?u.open(n.type,n.url,n.async,n.username,n.password):u.open(n.type,n.url,n.async),n.xhrFields)for(s in n.xhrFields)u[s]=n.xhrFields[s];n.mimeType&&u.overrideMimeType&&u.overrideMimeType(n.mimeType),n.crossDomain||i["X-Requested-With"]||(i["X-Requested-With"]="XMLHttpRequest");try{for(s in i)u.setRequestHeader(s,i[s])}catch(l){}u.send(n.hasContent&&n.data||null),r=function(e,i){var s,l,c,p;try{if(r&&(i||4===u.readyState))if(r=t,a&&(u.onreadystatechange=b.noop,$n&&delete Pn[a]),i)4!==u.readyState&&u.abort();else{p={},s=u.status,l=u.getAllResponseHeaders(),"string"==typeof u.responseText&&(p.text=u.responseText);try{c=u.statusText}catch(f){c=""}s||!n.isLocal||n.crossDomain?1223===s&&(s=204):s=p.text?200:404}}catch(d){i||o(-1,d)}p&&o(s,c,p,l)},n.async?4===u.readyState?setTimeout(r):(a=++Wn,$n&&(Pn||(Pn={},b(e).unload($n)),Pn[a]=r),u.onreadystatechange=r):r()},abort:function(){r&&r(t,!0)}}}});var Xn,Un,Vn=/^(?:toggle|show|hide)$/,Yn=RegExp("^(?:([+-])=|)("+x+")([a-z%]*)$","i"),Jn=/queueHooks$/,Gn=[nr],Qn={"*":[function(e,t){var n,r,i=this.createTween(e,t),o=Yn.exec(t),a=i.cur(),s=+a||0,u=1,l=20;if(o){if(n=+o[2],r=o[3]||(b.cssNumber[e]?"":"px"),"px"!==r&&s){s=b.css(i.elem,e,!0)||n||1;do u=u||".5",s/=u,b.style(i.elem,e,s+r);while(u!==(u=i.cur()/a)&&1!==u&&--l)}i.unit=r,i.start=s,i.end=o[1]?s+(o[1]+1)*n:n}return i}]};function Kn(){return setTimeout(function(){Xn=t}),Xn=b.now()}function Zn(e,t){b.each(t,function(t,n){var r=(Qn[t]||[]).concat(Qn["*"]),i=0,o=r.length;for(;o>i;i++)if(r[i].call(e,t,n))return})}function er(e,t,n){var r,i,o=0,a=Gn.length,s=b.Deferred().always(function(){delete u.elem}),u=function(){if(i)return!1;var t=Xn||Kn(),n=Math.max(0,l.startTime+l.duration-t),r=n/l.duration||0,o=1-r,a=0,u=l.tweens.length;for(;u>a;a++)l.tweens[a].run(o);return s.notifyWith(e,[l,o,n]),1>o&&u?n:(s.resolveWith(e,[l]),!1)},l=s.promise({elem:e,props:b.extend({},t),opts:b.extend(!0,{specialEasing:{}},n),originalProperties:t,originalOptions:n,startTime:Xn||Kn(),duration:n.duration,tweens:[],createTween:function(t,n){var r=b.Tween(e,l.opts,t,n,l.opts.specialEasing[t]||l.opts.easing);return l.tweens.push(r),r},stop:function(t){var n=0,r=t?l.tweens.length:0;if(i)return this;for(i=!0;r>n;n++)l.tweens[n].run(1);return t?s.resolveWith(e,[l,t]):s.rejectWith(e,[l,t]),this}}),c=l.props;for(tr(c,l.opts.specialEasing);a>o;o++)if(r=Gn[o].call(l,e,c,l.opts))return r;return Zn(l,c),b.isFunction(l.opts.start)&&l.opts.start.call(e,l),b.fx.timer(b.extend(u,{elem:e,anim:l,queue:l.opts.queue})),l.progress(l.opts.progress).done(l.opts.done,l.opts.complete).fail(l.opts.fail).always(l.opts.always)}function tr(e,t){var n,r,i,o,a;for(i in e)if(r=b.camelCase(i),o=t[r],n=e[i],b.isArray(n)&&(o=n[1],n=e[i]=n[0]),i!==r&&(e[r]=n,delete e[i]),a=b.cssHooks[r],a&&"expand"in a){n=a.expand(n),delete e[r];for(i in n)i in e||(e[i]=n[i],t[i]=o)}else t[r]=o}b.Animation=b.extend(er,{tweener:function(e,t){b.isFunction(e)?(t=e,e=["*"]):e=e.split(" ");var n,r=0,i=e.length;for(;i>r;r++)n=e[r],Qn[n]=Qn[n]||[],Qn[n].unshift(t)},prefilter:function(e,t){t?Gn.unshift(e):Gn.push(e)}});function nr(e,t,n){var r,i,o,a,s,u,l,c,p,f=this,d=e.style,h={},g=[],m=e.nodeType&&nn(e);n.queue||(c=b._queueHooks(e,"fx"),null==c.unqueued&&(c.unqueued=0,p=c.empty.fire,c.empty.fire=function(){c.unqueued||p()}),c.unqueued++,f.always(function(){f.always(function(){c.unqueued--,b.queue(e,"fx").length||c.empty.fire()})})),1===e.nodeType&&("height"in t||"width"in t)&&(n.overflow=[d.overflow,d.overflowX,d.overflowY],"inline"===b.css(e,"display")&&"none"===b.css(e,"float")&&(b.support.inlineBlockNeedsLayout&&"inline"!==un(e.nodeName)?d.zoom=1:d.display="inline-block")),n.overflow&&(d.overflow="hidden",b.support.shrinkWrapBlocks||f.always(function(){d.overflow=n.overflow[0],d.overflowX=n.overflow[1],d.overflowY=n.overflow[2]}));for(i in t)if(a=t[i],Vn.exec(a)){if(delete t[i],u=u||"toggle"===a,a===(m?"hide":"show"))continue;g.push(i)}if(o=g.length){s=b._data(e,"fxshow")||b._data(e,"fxshow",{}),"hidden"in s&&(m=s.hidden),u&&(s.hidden=!m),m?b(e).show():f.done(function(){b(e).hide()}),f.done(function(){var t;b._removeData(e,"fxshow");for(t in h)b.style(e,t,h[t])});for(i=0;o>i;i++)r=g[i],l=f.createTween(r,m?s[r]:0),h[r]=s[r]||b.style(e,r),r in s||(s[r]=l.start,m&&(l.end=l.start,l.start="width"===r||"height"===r?1:0))}}function rr(e,t,n,r,i){return new rr.prototype.init(e,t,n,r,i)}b.Tween=rr,rr.prototype={constructor:rr,init:function(e,t,n,r,i,o){this.elem=e,this.prop=n,this.easing=i||"swing",this.options=t,this.start=this.now=this.cur(),this.end=r,this.unit=o||(b.cssNumber[n]?"":"px")},cur:function(){var e=rr.propHooks[this.prop];return e&&e.get?e.get(this):rr.propHooks._default.get(this)},run:function(e){var t,n=rr.propHooks[this.prop];return this.pos=t=this.options.duration?b.easing[this.easing](e,this.options.duration*e,0,1,this.options.duration):e,this.now=(this.end-this.start)*t+this.start,this.options.step&&this.options.step.call(this.elem,this.now,this),n&&n.set?n.set(this):rr.propHooks._default.set(this),this}},rr.prototype.init.prototype=rr.prototype,rr.propHooks={_default:{get:function(e){var t;return null==e.elem[e.prop]||e.elem.style&&null!=e.elem.style[e.prop]?(t=b.css(e.elem,e.prop,""),t&&"auto"!==t?t:0):e.elem[e.prop]},set:function(e){b.fx.step[e.prop]?b.fx.step[e.prop](e):e.elem.style&&(null!=e.elem.style[b.cssProps[e.prop]]||b.cssHooks[e.prop])?b.style(e.elem,e.prop,e.now+e.unit):e.elem[e.prop]=e.now}}},rr.propHooks.scrollTop=rr.propHooks.scrollLeft={set:function(e){e.elem.nodeType&&e.elem.parentNode&&(e.elem[e.prop]=e.now)}},b.each(["toggle","show","hide"],function(e,t){var n=b.fn[t];b.fn[t]=function(e,r,i){return null==e||"boolean"==typeof e?n.apply(this,arguments):this.animate(ir(t,!0),e,r,i)}}),b.fn.extend({fadeTo:function(e,t,n,r){return this.filter(nn).css("opacity",0).show().end().animate({opacity:t},e,n,r)},animate:function(e,t,n,r){var i=b.isEmptyObject(e),o=b.speed(t,n,r),a=function(){var t=er(this,b.extend({},e),o);a.finish=function(){t.stop(!0)},(i||b._data(this,"finish"))&&t.stop(!0)};return a.finish=a,i||o.queue===!1?this.each(a):this.queue(o.queue,a)},stop:function(e,n,r){var i=function(e){var t=e.stop;delete e.stop,t(r)};return"string"!=typeof e&&(r=n,n=e,e=t),n&&e!==!1&&this.queue(e||"fx",[]),this.each(function(){var t=!0,n=null!=e&&e+"queueHooks",o=b.timers,a=b._data(this);if(n)a[n]&&a[n].stop&&i(a[n]);else for(n in a)a[n]&&a[n].stop&&Jn.test(n)&&i(a[n]);for(n=o.length;n--;)o[n].elem!==this||null!=e&&o[n].queue!==e||(o[n].anim.stop(r),t=!1,o.splice(n,1));(t||!r)&&b.dequeue(this,e)})},finish:function(e){return e!==!1&&(e=e||"fx"),this.each(function(){var t,n=b._data(this),r=n[e+"queue"],i=n[e+"queueHooks"],o=b.timers,a=r?r.length:0;for(n.finish=!0,b.queue(this,e,[]),i&&i.cur&&i.cur.finish&&i.cur.finish.call(this),t=o.length;t--;)o[t].elem===this&&o[t].queue===e&&(o[t].anim.stop(!0),o.splice(t,1));for(t=0;a>t;t++)r[t]&&r[t].finish&&r[t].finish.call(this);delete n.finish})}});function ir(e,t){var n,r={height:e},i=0;for(t=t?1:0;4>i;i+=2-t)n=Zt[i],r["margin"+n]=r["padding"+n]=e;return t&&(r.opacity=r.width=e),r}b.each({slideDown:ir("show"),slideUp:ir("hide"),slideToggle:ir("toggle"),fadeIn:{opacity:"show"},fadeOut:{opacity:"hide"},fadeToggle:{opacity:"toggle"}},function(e,t){b.fn[e]=function(e,n,r){return this.animate(t,e,n,r)}}),b.speed=function(e,t,n){var r=e&&"object"==typeof e?b.extend({},e):{complete:n||!n&&t||b.isFunction(e)&&e,duration:e,easing:n&&t||t&&!b.isFunction(t)&&t};return r.duration=b.fx.off?0:"number"==typeof r.duration?r.duration:r.duration in b.fx.speeds?b.fx.speeds[r.duration]:b.fx.speeds._default,(null==r.queue||r.queue===!0)&&(r.queue="fx"),r.old=r.complete,r.complete=function(){b.isFunction(r.old)&&r.old.call(this),r.queue&&b.dequeue(this,r.queue)},r},b.easing={linear:function(e){return e},swing:function(e){return.5-Math.cos(e*Math.PI)/2}},b.timers=[],b.fx=rr.prototype.init,b.fx.tick=function(){var e,n=b.timers,r=0;for(Xn=b.now();n.length>r;r++)e=n[r],e()||n[r]!==e||n.splice(r--,1);n.length||b.fx.stop(),Xn=t},b.fx.timer=function(e){e()&&b.timers.push(e)&&b.fx.start()},b.fx.interval=13,b.fx.start=function(){Un||(Un=setInterval(b.fx.tick,b.fx.interval))},b.fx.stop=function(){clearInterval(Un),Un=null},b.fx.speeds={slow:600,fast:200,_default:400},b.fx.step={},b.expr&&b.expr.filters&&(b.expr.filters.animated=function(e){return b.grep(b.timers,function(t){return e===t.elem}).length}),b.fn.offset=function(e){if(arguments.length)return e===t?this:this.each(function(t){b.offset.setOffset(this,e,t)});var n,r,o={top:0,left:0},a=this[0],s=a&&a.ownerDocument;if(s)return n=s.documentElement,b.contains(n,a)?(typeof a.getBoundingClientRect!==i&&(o=a.getBoundingClientRect()),r=or(s),{top:o.top+(r.pageYOffset||n.scrollTop)-(n.clientTop||0),left:o.left+(r.pageXOffset||n.scrollLeft)-(n.clientLeft||0)}):o},b.offset={setOffset:function(e,t,n){var r=b.css(e,"position");"static"===r&&(e.style.position="relative");var i=b(e),o=i.offset(),a=b.css(e,"top"),s=b.css(e,"left"),u=("absolute"===r||"fixed"===r)&&b.inArray("auto",[a,s])>-1,l={},c={},p,f;u?(c=i.position(),p=c.top,f=c.left):(p=parseFloat(a)||0,f=parseFloat(s)||0),b.isFunction(t)&&(t=t.call(e,n,o)),null!=t.top&&(l.top=t.top-o.top+p),null!=t.left&&(l.left=t.left-o.left+f),"using"in t?t.using.call(e,l):i.css(l)}},b.fn.extend({position:function(){if(this[0]){var e,t,n={top:0,left:0},r=this[0];return"fixed"===b.css(r,"position")?t=r.getBoundingClientRect():(e=this.offsetParent(),t=this.offset(),b.nodeName(e[0],"html")||(n=e.offset()),n.top+=b.css(e[0],"borderTopWidth",!0),n.left+=b.css(e[0],"borderLeftWidth",!0)),{top:t.top-n.top-b.css(r,"marginTop",!0),left:t.left-n.left-b.css(r,"marginLeft",!0)}}},offsetParent:function(){return this.map(function(){var e=this.offsetParent||o.documentElement;while(e&&!b.nodeName(e,"html")&&"static"===b.css(e,"position"))e=e.offsetParent;return e||o.documentElement})}}),b.each({scrollLeft:"pageXOffset",scrollTop:"pageYOffset"},function(e,n){var r=/Y/.test(n);b.fn[e]=function(i){return b.access(this,function(e,i,o){var a=or(e);return o===t?a?n in a?a[n]:a.document.documentElement[i]:e[i]:(a?a.scrollTo(r?b(a).scrollLeft():o,r?o:b(a).scrollTop()):e[i]=o,t)},e,i,arguments.length,null)}});function or(e){return b.isWindow(e)?e:9===e.nodeType?e.defaultView||e.parentWindow:!1}b.each({Height:"height",Width:"width"},function(e,n){b.each({padding:"inner"+e,content:n,"":"outer"+e},function(r,i){b.fn[i]=function(i,o){var a=arguments.length&&(r||"boolean"!=typeof i),s=r||(i===!0||o===!0?"margin":"border");return b.access(this,function(n,r,i){var o;return b.isWindow(n)?n.document.documentElement["client"+e]:9===n.nodeType?(o=n.documentElement,Math.max(n.body["scroll"+e],o["scroll"+e],n.body["offset"+e],o["offset"+e],o["client"+e])):i===t?b.css(n,r,s):b.style(n,r,i,s)},n,a?i:t,a,null)}})}),e.jQuery=e.$=b,"function"==typeof define&&define.amd&&define.amd.jQuery&&define("jquery",[],function(){return b})})(window);
\ No newline at end of file
+}b.merge(d,s.childNodes),s.textContent="";while(s.firstChild)s.removeChild(s.firstChild);s=f.lastChild}else d.push(t.createTextNode(o));s&&f.removeChild(s),b.support.appendChecked||b.grep(Ot(d,"input"),Bt),h=0;while(o=d[h++])if((!r||-1===b.inArray(o,r))&&(a=b.contains(o.ownerDocument,o),s=Ot(f.appendChild(o),"script"),a&&Mt(s),n)){i=0;while(o=s[i++])kt.test(o.type||"")&&n.push(o)}return s=null,f},cleanData:function(e,t){var n,r,o,a,s=0,u=b.expando,l=b.cache,p=b.support.deleteExpando,f=b.event.special;for(;null!=(n=e[s]);s++)if((t||b.acceptData(n))&&(o=n[u],a=o&&l[o])){if(a.events)for(r in a.events)f[r]?b.event.remove(n,r):b.removeEvent(n,r,a.handle);l[o]&&(delete l[o],p?delete n[u]:typeof n.removeAttribute!==i?n.removeAttribute(u):n[u]=null,c.push(o))}}});var Pt,Rt,Wt,$t=/alpha\([^)]*\)/i,It=/opacity\s*=\s*([^)]*)/,zt=/^(top|right|bottom|left)$/,Xt=/^(none|table(?!-c[ea]).+)/,Ut=/^margin/,Vt=RegExp("^("+x+")(.*)$","i"),Yt=RegExp("^("+x+")(?!px)[a-z%]+$","i"),Jt=RegExp("^([+-])=("+x+")","i"),Gt={BODY:"block"},Qt={position:"absolute",visibility:"hidden",display:"block"},Kt={letterSpacing:0,fontWeight:400},Zt=["Top","Right","Bottom","Left"],en=["Webkit","O","Moz","ms"];function tn(e,t){if(t in e)return t;var n=t.charAt(0).toUpperCase()+t.slice(1),r=t,i=en.length;while(i--)if(t=en[i]+n,t in e)return t;return r}function nn(e,t){return e=t||e,"none"===b.css(e,"display")||!b.contains(e.ownerDocument,e)}function rn(e,t){var n,r,i,o=[],a=0,s=e.length;for(;s>a;a++)r=e[a],r.style&&(o[a]=b._data(r,"olddisplay"),n=r.style.display,t?(o[a]||"none"!==n||(r.style.display=""),""===r.style.display&&nn(r)&&(o[a]=b._data(r,"olddisplay",un(r.nodeName)))):o[a]||(i=nn(r),(n&&"none"!==n||!i)&&b._data(r,"olddisplay",i?n:b.css(r,"display"))));for(a=0;s>a;a++)r=e[a],r.style&&(t&&"none"!==r.style.display&&""!==r.style.display||(r.style.display=t?o[a]||"":"none"));return e}b.fn.extend({css:function(e,n){return b.access(this,function(e,n,r){var i,o,a={},s=0;if(b.isArray(n)){for(o=Rt(e),i=n.length;i>s;s++)a[n[s]]=b.css(e,n[s],!1,o);return a}return r!==t?b.style(e,n,r):b.css(e,n)},e,n,arguments.length>1)},show:function(){return rn(this,!0)},hide:function(){return rn(this)},toggle:function(e){var t="boolean"==typeof e;return this.each(function(){(t?e:nn(this))?b(this).show():b(this).hide()})}}),b.extend({cssHooks:{opacity:{get:function(e,t){if(t){var n=Wt(e,"opacity");return""===n?"1":n}}}},cssNumber:{columnCount:!0,fillOpacity:!0,fontWeight:!0,lineHeight:!0,opacity:!0,orphans:!0,widows:!0,zIndex:!0,zoom:!0},cssProps:{"float":b.support.cssFloat?"cssFloat":"styleFloat"},style:function(e,n,r,i){if(e&&3!==e.nodeType&&8!==e.nodeType&&e.style){var o,a,s,u=b.camelCase(n),l=e.style;if(n=b.cssProps[u]||(b.cssProps[u]=tn(l,u)),s=b.cssHooks[n]||b.cssHooks[u],r===t)return s&&"get"in s&&(o=s.get(e,!1,i))!==t?o:l[n];if(a=typeof r,"string"===a&&(o=Jt.exec(r))&&(r=(o[1]+1)*o[2]+parseFloat(b.css(e,n)),a="number"),!(null==r||"number"===a&&isNaN(r)||("number"!==a||b.cssNumber[u]||(r+="px"),b.support.clearCloneStyle||""!==r||0!==n.indexOf("background")||(l[n]="inherit"),s&&"set"in s&&(r=s.set(e,r,i))===t)))try{l[n]=r}catch(c){}}},css:function(e,n,r,i){var o,a,s,u=b.camelCase(n);return n=b.cssProps[u]||(b.cssProps[u]=tn(e.style,u)),s=b.cssHooks[n]||b.cssHooks[u],s&&"get"in s&&(a=s.get(e,!0,r)),a===t&&(a=Wt(e,n,i)),"normal"===a&&n in Kt&&(a=Kt[n]),""===r||r?(o=parseFloat(a),r===!0||b.isNumeric(o)?o||0:a):a},swap:function(e,t,n,r){var i,o,a={};for(o in t)a[o]=e.style[o],e.style[o]=t[o];i=n.apply(e,r||[]);for(o in t)e.style[o]=a[o];return i}}),e.getComputedStyle?(Rt=function(t){return e.getComputedStyle(t,null)},Wt=function(e,n,r){var i,o,a,s=r||Rt(e),u=s?s.getPropertyValue(n)||s[n]:t,l=e.style;return s&&(""!==u||b.contains(e.ownerDocument,e)||(u=b.style(e,n)),Yt.test(u)&&Ut.test(n)&&(i=l.width,o=l.minWidth,a=l.maxWidth,l.minWidth=l.maxWidth=l.width=u,u=s.width,l.width=i,l.minWidth=o,l.maxWidth=a)),u}):o.documentElement.currentStyle&&(Rt=function(e){return e.currentStyle},Wt=function(e,n,r){var i,o,a,s=r||Rt(e),u=s?s[n]:t,l=e.style;return null==u&&l&&l[n]&&(u=l[n]),Yt.test(u)&&!zt.test(n)&&(i=l.left,o=e.runtimeStyle,a=o&&o.left,a&&(o.left=e.currentStyle.left),l.left="fontSize"===n?"1em":u,u=l.pixelLeft+"px",l.left=i,a&&(o.left=a)),""===u?"auto":u});function on(e,t,n){var r=Vt.exec(t);return r?Math.max(0,r[1]-(n||0))+(r[2]||"px"):t}function an(e,t,n,r,i){var o=n===(r?"border":"content")?4:"width"===t?1:0,a=0;for(;4>o;o+=2)"margin"===n&&(a+=b.css(e,n+Zt[o],!0,i)),r?("content"===n&&(a-=b.css(e,"padding"+Zt[o],!0,i)),"margin"!==n&&(a-=b.css(e,"border"+Zt[o]+"Width",!0,i))):(a+=b.css(e,"padding"+Zt[o],!0,i),"padding"!==n&&(a+=b.css(e,"border"+Zt[o]+"Width",!0,i)));return a}function sn(e,t,n){var r=!0,i="width"===t?e.offsetWidth:e.offsetHeight,o=Rt(e),a=b.support.boxSizing&&"border-box"===b.css(e,"boxSizing",!1,o);if(0>=i||null==i){if(i=Wt(e,t,o),(0>i||null==i)&&(i=e.style[t]),Yt.test(i))return i;r=a&&(b.support.boxSizingReliable||i===e.style[t]),i=parseFloat(i)||0}return i+an(e,t,n||(a?"border":"content"),r,o)+"px"}function un(e){var t=o,n=Gt[e];return n||(n=ln(e,t),"none"!==n&&n||(Pt=(Pt||b("<iframe frameborder='0' width='0' height='0'/>").css("cssText","display:block !important")).appendTo(t.documentElement),t=(Pt[0].contentWindow||Pt[0].contentDocument).document,t.write("<!doctype html><html><body>"),t.close(),n=ln(e,t),Pt.detach()),Gt[e]=n),n}function ln(e,t){var n=b(t.createElement(e)).appendTo(t.body),r=b.css(n[0],"display");return n.remove(),r}b.each(["height","width"],function(e,n){b.cssHooks[n]={get:function(e,r,i){return r?0===e.offsetWidth&&Xt.test(b.css(e,"display"))?b.swap(e,Qt,function(){return sn(e,n,i)}):sn(e,n,i):t},set:function(e,t,r){var i=r&&Rt(e);return on(e,t,r?an(e,n,r,b.support.boxSizing&&"border-box"===b.css(e,"boxSizing",!1,i),i):0)}}}),b.support.opacity||(b.cssHooks.opacity={get:function(e,t){return It.test((t&&e.currentStyle?e.currentStyle.filter:e.style.filter)||"")?.01*parseFloat(RegExp.$1)+"":t?"1":""},set:function(e,t){var n=e.style,r=e.currentStyle,i=b.isNumeric(t)?"alpha(opacity="+100*t+")":"",o=r&&r.filter||n.filter||"";n.zoom=1,(t>=1||""===t)&&""===b.trim(o.replace($t,""))&&n.removeAttribute&&(n.removeAttribute("filter"),""===t||r&&!r.filter)||(n.filter=$t.test(o)?o.replace($t,i):o+" "+i)}}),b(function(){b.support.reliableMarginRight||(b.cssHooks.marginRight={get:function(e,n){return n?b.swap(e,{display:"inline-block"},Wt,[e,"marginRight"]):t}}),!b.support.pixelPosition&&b.fn.position&&b.each(["top","left"],function(e,n){b.cssHooks[n]={get:function(e,r){return r?(r=Wt(e,n),Yt.test(r)?b(e).position()[n]+"px":r):t}}})}),b.expr&&b.expr.filters&&(b.expr.filters.hidden=function(e){return 0>=e.offsetWidth&&0>=e.offsetHeight||!b.support.reliableHiddenOffsets&&"none"===(e.style&&e.style.display||b.css(e,"display"))},b.expr.filters.visible=function(e){return!b.expr.filters.hidden(e)}),b.each({margin:"",padding:"",border:"Width"},function(e,t){b.cssHooks[e+t]={expand:function(n){var r=0,i={},o="string"==typeof n?n.split(" "):[n];for(;4>r;r++)i[e+Zt[r]+t]=o[r]||o[r-2]||o[0];return i}},Ut.test(e)||(b.cssHooks[e+t].set=on)});var cn=/%20/g,pn=/\[\]$/,fn=/\r?\n/g,dn=/^(?:submit|button|image|reset|file)$/i,hn=/^(?:input|select|textarea|keygen)/i;b.fn.extend({serialize:function(){return b.param(this.serializeArray())},serializeArray:function(){return this.map(function(){var e=b.prop(this,"elements");return e?b.makeArray(e):this}).filter(function(){var e=this.type;return this.name&&!b(this).is(":disabled")&&hn.test(this.nodeName)&&!dn.test(e)&&(this.checked||!Nt.test(e))}).map(function(e,t){var n=b(this).val();return null==n?null:b.isArray(n)?b.map(n,function(e){return{name:t.name,value:e.replace(fn,"\r\n")}}):{name:t.name,value:n.replace(fn,"\r\n")}}).get()}}),b.param=function(e,n){var r,i=[],o=function(e,t){t=b.isFunction(t)?t():null==t?"":t,i[i.length]=encodeURIComponent(e)+"="+encodeURIComponent(t)};if(n===t&&(n=b.ajaxSettings&&b.ajaxSettings.traditional),b.isArray(e)||e.jquery&&!b.isPlainObject(e))b.each(e,function(){o(this.name,this.value)});else for(r in e)gn(r,e[r],n,o);return i.join("&").replace(cn,"+")};function gn(e,t,n,r){var i;if(b.isArray(t))b.each(t,function(t,i){n||pn.test(e)?r(e,i):gn(e+"["+("object"==typeof i?t:"")+"]",i,n,r)});else if(n||"object"!==b.type(t))r(e,t);else for(i in t)gn(e+"["+i+"]",t[i],n,r)}b.each("blur focus focusin focusout load resize scroll unload click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup error contextmenu".split(" "),function(e,t){b.fn[t]=function(e,n){return arguments.length>0?this.on(t,null,e,n):this.trigger(t)}}),b.fn.hover=function(e,t){return this.mouseenter(e).mouseleave(t||e)};var mn,yn,vn=b.now(),bn=/\?/,xn=/#.*$/,wn=/([?&])_=[^&]*/,Tn=/^(.*?):[ \t]*([^\r\n]*)\r?$/gm,Nn=/^(?:about|app|app-storage|.+-extension|file|res|widget):$/,Cn=/^(?:GET|HEAD)$/,kn=/^\/\//,En=/^([\w.+-]+:)(?:\/\/([^\/?#:]*)(?::(\d+)|)|)/,Sn=b.fn.load,An={},jn={},Dn="*/".concat("*");try{yn=a.href}catch(Ln){yn=o.createElement("a"),yn.href="",yn=yn.href}mn=En.exec(yn.toLowerCase())||[];function Hn(e){return function(t,n){"string"!=typeof t&&(n=t,t="*");var r,i=0,o=t.toLowerCase().match(w)||[];if(b.isFunction(n))while(r=o[i++])"+"===r[0]?(r=r.slice(1)||"*",(e[r]=e[r]||[]).unshift(n)):(e[r]=e[r]||[]).push(n)}}function qn(e,n,r,i){var o={},a=e===jn;function s(u){var l;return o[u]=!0,b.each(e[u]||[],function(e,u){var c=u(n,r,i);return"string"!=typeof c||a||o[c]?a?!(l=c):t:(n.dataTypes.unshift(c),s(c),!1)}),l}return s(n.dataTypes[0])||!o["*"]&&s("*")}function Mn(e,n){var r,i,o=b.ajaxSettings.flatOptions||{};for(i in n)n[i]!==t&&((o[i]?e:r||(r={}))[i]=n[i]);return r&&b.extend(!0,e,r),e}b.fn.load=function(e,n,r){if("string"!=typeof e&&Sn)return Sn.apply(this,arguments);var i,o,a,s=this,u=e.indexOf(" ");return u>=0&&(i=e.slice(u,e.length),e=e.slice(0,u)),b.isFunction(n)?(r=n,n=t):n&&"object"==typeof n&&(a="POST"),s.length>0&&b.ajax({url:e,type:a,dataType:"html",data:n}).done(function(e){o=arguments,s.html(i?b("<div>").append(b.parseHTML(e)).find(i):e)}).complete(r&&function(e,t){s.each(r,o||[e.responseText,t,e])}),this},b.each(["ajaxStart","ajaxStop","ajaxComplete","ajaxError","ajaxSuccess","ajaxSend"],function(e,t){b.fn[t]=function(e){return this.on(t,e)}}),b.each(["get","post"],function(e,n){b[n]=function(e,r,i,o){return b.isFunction(r)&&(o=o||i,i=r,r=t),b.ajax({url:e,type:n,dataType:o,data:r,success:i})}}),b.extend({active:0,lastModified:{},etag:{},ajaxSettings:{url:yn,type:"GET",isLocal:Nn.test(mn[1]),global:!0,processData:!0,async:!0,contentType:"application/x-www-form-urlencoded; charset=UTF-8",accepts:{"*":Dn,text:"text/plain",html:"text/html",xml:"application/xml, text/xml",json:"application/json, text/javascript"},contents:{xml:/xml/,html:/html/,json:/json/},responseFields:{xml:"responseXML",text:"responseText"},converters:{"* text":e.String,"text html":!0,"text json":b.parseJSON,"text xml":b.parseXML},flatOptions:{url:!0,context:!0}},ajaxSetup:function(e,t){return t?Mn(Mn(e,b.ajaxSettings),t):Mn(b.ajaxSettings,e)},ajaxPrefilter:Hn(An),ajaxTransport:Hn(jn),ajax:function(e,n){"object"==typeof e&&(n=e,e=t),n=n||{};var r,i,o,a,s,u,l,c,p=b.ajaxSetup({},n),f=p.context||p,d=p.context&&(f.nodeType||f.jquery)?b(f):b.event,h=b.Deferred(),g=b.Callbacks("once memory"),m=p.statusCode||{},y={},v={},x=0,T="canceled",N={readyState:0,getResponseHeader:function(e){var t;if(2===x){if(!c){c={};while(t=Tn.exec(a))c[t[1].toLowerCase()]=t[2]}t=c[e.toLowerCase()]}return null==t?null:t},getAllResponseHeaders:function(){return 2===x?a:null},setRequestHeader:function(e,t){var n=e.toLowerCase();return x||(e=v[n]=v[n]||e,y[e]=t),this},overrideMimeType:function(e){return x||(p.mimeType=e),this},statusCode:function(e){var t;if(e)if(2>x)for(t in e)m[t]=[m[t],e[t]];else N.always(e[N.status]);return this},abort:function(e){var t=e||T;return l&&l.abort(t),k(0,t),this}};if(h.promise(N).complete=g.add,N.success=N.done,N.error=N.fail,p.url=((e||p.url||yn)+"").replace(xn,"").replace(kn,mn[1]+"//"),p.type=n.method||n.type||p.method||p.type,p.dataTypes=b.trim(p.dataType||"*").toLowerCase().match(w)||[""],null==p.crossDomain&&(r=En.exec(p.url.toLowerCase()),p.crossDomain=!(!r||r[1]===mn[1]&&r[2]===mn[2]&&(r[3]||("http:"===r[1]?80:443))==(mn[3]||("http:"===mn[1]?80:443)))),p.data&&p.processData&&"string"!=typeof p.data&&(p.data=b.param(p.data,p.traditional)),qn(An,p,n,N),2===x)return N;u=p.global,u&&0===b.active++&&b.event.trigger("ajaxStart"),p.type=p.type.toUpperCase(),p.hasContent=!Cn.test(p.type),o=p.url,p.hasContent||(p.data&&(o=p.url+=(bn.test(o)?"&":"?")+p.data,delete p.data),p.cache===!1&&(p.url=wn.test(o)?o.replace(wn,"$1_="+vn++):o+(bn.test(o)?"&":"?")+"_="+vn++)),p.ifModified&&(b.lastModified[o]&&N.setRequestHeader("If-Modified-Since",b.lastModified[o]),b.etag[o]&&N.setRequestHeader("If-None-Match",b.etag[o])),(p.data&&p.hasContent&&p.contentType!==!1||n.contentType)&&N.setRequestHeader("Content-Type",p.contentType),N.setRequestHeader("Accept",p.dataTypes[0]&&p.accepts[p.dataTypes[0]]?p.accepts[p.dataTypes[0]]+("*"!==p.dataTypes[0]?", "+Dn+"; q=0.01":""):p.accepts["*"]);for(i in p.headers)N.setRequestHeader(i,p.headers[i]);if(p.beforeSend&&(p.beforeSend.call(f,N,p)===!1||2===x))return N.abort();T="abort";for(i in{success:1,error:1,complete:1})N[i](p[i]);if(l=qn(jn,p,n,N)){N.readyState=1,u&&d.trigger("ajaxSend",[N,p]),p.async&&p.timeout>0&&(s=setTimeout(function(){N.abort("timeout")},p.timeout));try{x=1,l.send(y,k)}catch(C){if(!(2>x))throw C;k(-1,C)}}else k(-1,"No Transport");function k(e,n,r,i){var c,y,v,w,T,C=n;2!==x&&(x=2,s&&clearTimeout(s),l=t,a=i||"",N.readyState=e>0?4:0,r&&(w=_n(p,N,r)),e>=200&&300>e||304===e?(p.ifModified&&(T=N.getResponseHeader("Last-Modified"),T&&(b.lastModified[o]=T),T=N.getResponseHeader("etag"),T&&(b.etag[o]=T)),204===e?(c=!0,C="nocontent"):304===e?(c=!0,C="notmodified"):(c=Fn(p,w),C=c.state,y=c.data,v=c.error,c=!v)):(v=C,(e||!C)&&(C="error",0>e&&(e=0))),N.status=e,N.statusText=(n||C)+"",c?h.resolveWith(f,[y,C,N]):h.rejectWith(f,[N,C,v]),N.statusCode(m),m=t,u&&d.trigger(c?"ajaxSuccess":"ajaxError",[N,p,c?y:v]),g.fireWith(f,[N,C]),u&&(d.trigger("ajaxComplete",[N,p]),--b.active||b.event.trigger("ajaxStop")))}return N},getScript:function(e,n){return b.get(e,t,n,"script")},getJSON:function(e,t,n){return b.get(e,t,n,"json")}});function _n(e,n,r){var i,o,a,s,u=e.contents,l=e.dataTypes,c=e.responseFields;for(s in c)s in r&&(n[c[s]]=r[s]);while("*"===l[0])l.shift(),o===t&&(o=e.mimeType||n.getResponseHeader("Content-Type"));if(o)for(s in u)if(u[s]&&u[s].test(o)){l.unshift(s);break}if(l[0]in r)a=l[0];else{for(s in r){if(!l[0]||e.converters[s+" "+l[0]]){a=s;break}i||(i=s)}a=a||i}return a?(a!==l[0]&&l.unshift(a),r[a]):t}function Fn(e,t){var n,r,i,o,a={},s=0,u=e.dataTypes.slice(),l=u[0];if(e.dataFilter&&(t=e.dataFilter(t,e.dataType)),u[1])for(i in e.converters)a[i.toLowerCase()]=e.converters[i];for(;r=u[++s];)if("*"!==r){if("*"!==l&&l!==r){if(i=a[l+" "+r]||a["* "+r],!i)for(n in a)if(o=n.split(" "),o[1]===r&&(i=a[l+" "+o[0]]||a["* "+o[0]])){i===!0?i=a[n]:a[n]!==!0&&(r=o[0],u.splice(s--,0,r));break}if(i!==!0)if(i&&e["throws"])t=i(t);else try{t=i(t)}catch(c){return{state:"parsererror",error:i?c:"No conversion from "+l+" to "+r}}}l=r}return{state:"success",data:t}}b.ajaxSetup({accepts:{script:"text/javascript, application/javascript, application/ecmascript, application/x-ecmascript"},contents:{script:/(?:java|ecma)script/},converters:{"text script":function(e){return b.globalEval(e),e}}}),b.ajaxPrefilter("script",function(e){e.cache===t&&(e.cache=!1),e.crossDomain&&(e.type="GET",e.global=!1)}),b.ajaxTransport("script",function(e){if(e.crossDomain){var n,r=o.head||b("head")[0]||o.documentElement;return{send:function(t,i){n=o.createElement("script"),n.async=!0,e.scriptCharset&&(n.charset=e.scriptCharset),n.src=e.url,n.onload=n.onreadystatechange=function(e,t){(t||!n.readyState||/loaded|complete/.test(n.readyState))&&(n.onload=n.onreadystatechange=null,n.parentNode&&n.parentNode.removeChild(n),n=null,t||i(200,"success"))},r.insertBefore(n,r.firstChild)},abort:function(){n&&n.onload(t,!0)}}}});var On=[],Bn=/(=)\?(?=&|$)|\?\?/;b.ajaxSetup({jsonp:"callback",jsonpCallback:function(){var e=On.pop()||b.expando+"_"+vn++;return this[e]=!0,e}}),b.ajaxPrefilter("json jsonp",function(n,r,i){var o,a,s,u=n.jsonp!==!1&&(Bn.test(n.url)?"url":"string"==typeof n.data&&!(n.contentType||"").indexOf("application/x-www-form-urlencoded")&&Bn.test(n.data)&&"data");return u||"jsonp"===n.dataTypes[0]?(o=n.jsonpCallback=b.isFunction(n.jsonpCallback)?n.jsonpCallback():n.jsonpCallback,u?n[u]=n[u].replace(Bn,"$1"+o):n.jsonp!==!1&&(n.url+=(bn.test(n.url)?"&":"?")+n.jsonp+"="+o),n.converters["script json"]=function(){return s||b.error(o+" was not called"),s[0]},n.dataTypes[0]="json",a=e[o],e[o]=function(){s=arguments},i.always(function(){e[o]=a,n[o]&&(n.jsonpCallback=r.jsonpCallback,On.push(o)),s&&b.isFunction(a)&&a(s[0]),s=a=t}),"script"):t});var Pn,Rn,Wn=0,$n=e.ActiveXObject&&function(){var e;for(e in Pn)Pn[e](t,!0)};function In(){try{return new e.XMLHttpRequest}catch(t){}}function zn(){try{return new e.ActiveXObject("Microsoft.XMLHTTP")}catch(t){}}b.ajaxSettings.xhr=e.ActiveXObject?function(){return!this.isLocal&&In()||zn()}:In,Rn=b.ajaxSettings.xhr(),b.support.cors=!!Rn&&"withCredentials"in Rn,Rn=b.support.ajax=!!Rn,Rn&&b.ajaxTransport(function(n){if(!n.crossDomain||b.support.cors){var r;return{send:function(i,o){var a,s,u=n.xhr();if(n.username?u.open(n.type,n.url,n.async,n.username,n.password):u.open(n.type,n.url,n.async),n.xhrFields)for(s in n.xhrFields)u[s]=n.xhrFields[s];n.mimeType&&u.overrideMimeType&&u.overrideMimeType(n.mimeType),n.crossDomain||i["X-Requested-With"]||(i["X-Requested-With"]="XMLHttpRequest");try{for(s in i)u.setRequestHeader(s,i[s])}catch(l){}u.send(n.hasContent&&n.data||null),r=function(e,i){var s,l,c,p;try{if(r&&(i||4===u.readyState))if(r=t,a&&(u.onreadystatechange=b.noop,$n&&delete Pn[a]),i)4!==u.readyState&&u.abort();else{p={},s=u.status,l=u.getAllResponseHeaders(),"string"==typeof u.responseText&&(p.text=u.responseText);try{c=u.statusText}catch(f){c=""}s||!n.isLocal||n.crossDomain?1223===s&&(s=204):s=p.text?200:404}}catch(d){i||o(-1,d)}p&&o(s,c,p,l)},n.async?4===u.readyState?setTimeout(r):(a=++Wn,$n&&(Pn||(Pn={},b(e).unload($n)),Pn[a]=r),u.onreadystatechange=r):r()},abort:function(){r&&r(t,!0)}}}});var Xn,Un,Vn=/^(?:toggle|show|hide)$/,Yn=RegExp("^(?:([+-])=|)("+x+")([a-z%]*)$","i"),Jn=/queueHooks$/,Gn=[nr],Qn={"*":[function(e,t){var n,r,i=this.createTween(e,t),o=Yn.exec(t),a=i.cur(),s=+a||0,u=1,l=20;if(o){if(n=+o[2],r=o[3]||(b.cssNumber[e]?"":"px"),"px"!==r&&s){s=b.css(i.elem,e,!0)||n||1;do u=u||".5",s/=u,b.style(i.elem,e,s+r);while(u!==(u=i.cur()/a)&&1!==u&&--l)}i.unit=r,i.start=s,i.end=o[1]?s+(o[1]+1)*n:n}return i}]};function Kn(){return setTimeout(function(){Xn=t}),Xn=b.now()}function Zn(e,t){b.each(t,function(t,n){var r=(Qn[t]||[]).concat(Qn["*"]),i=0,o=r.length;for(;o>i;i++)if(r[i].call(e,t,n))return})}function er(e,t,n){var r,i,o=0,a=Gn.length,s=b.Deferred().always(function(){delete u.elem}),u=function(){if(i)return!1;var t=Xn||Kn(),n=Math.max(0,l.startTime+l.duration-t),r=n/l.duration||0,o=1-r,a=0,u=l.tweens.length;for(;u>a;a++)l.tweens[a].run(o);return s.notifyWith(e,[l,o,n]),1>o&&u?n:(s.resolveWith(e,[l]),!1)},l=s.promise({elem:e,props:b.extend({},t),opts:b.extend(!0,{specialEasing:{}},n),originalProperties:t,originalOptions:n,startTime:Xn||Kn(),duration:n.duration,tweens:[],createTween:function(t,n){var r=b.Tween(e,l.opts,t,n,l.opts.specialEasing[t]||l.opts.easing);return l.tweens.push(r),r},stop:function(t){var n=0,r=t?l.tweens.length:0;if(i)return this;for(i=!0;r>n;n++)l.tweens[n].run(1);return t?s.resolveWith(e,[l,t]):s.rejectWith(e,[l,t]),this}}),c=l.props;for(tr(c,l.opts.specialEasing);a>o;o++)if(r=Gn[o].call(l,e,c,l.opts))return r;return Zn(l,c),b.isFunction(l.opts.start)&&l.opts.start.call(e,l),b.fx.timer(b.extend(u,{elem:e,anim:l,queue:l.opts.queue})),l.progress(l.opts.progress).done(l.opts.done,l.opts.complete).fail(l.opts.fail).always(l.opts.always)}function tr(e,t){var n,r,i,o,a;for(i in e)if(r=b.camelCase(i),o=t[r],n=e[i],b.isArray(n)&&(o=n[1],n=e[i]=n[0]),i!==r&&(e[r]=n,delete e[i]),a=b.cssHooks[r],a&&"expand"in a){n=a.expand(n),delete e[r];for(i in n)i in e||(e[i]=n[i],t[i]=o)}else t[r]=o}b.Animation=b.extend(er,{tweener:function(e,t){b.isFunction(e)?(t=e,e=["*"]):e=e.split(" ");var n,r=0,i=e.length;for(;i>r;r++)n=e[r],Qn[n]=Qn[n]||[],Qn[n].unshift(t)},prefilter:function(e,t){t?Gn.unshift(e):Gn.push(e)}});function nr(e,t,n){var r,i,o,a,s,u,l,c,p,f=this,d=e.style,h={},g=[],m=e.nodeType&&nn(e);n.queue||(c=b._queueHooks(e,"fx"),null==c.unqueued&&(c.unqueued=0,p=c.empty.fire,c.empty.fire=function(){c.unqueued||p()}),c.unqueued++,f.always(function(){f.always(function(){c.unqueued--,b.queue(e,"fx").length||c.empty.fire()})})),1===e.nodeType&&("height"in t||"width"in t)&&(n.overflow=[d.overflow,d.overflowX,d.overflowY],"inline"===b.css(e,"display")&&"none"===b.css(e,"float")&&(b.support.inlineBlockNeedsLayout&&"inline"!==un(e.nodeName)?d.zoom=1:d.display="inline-block")),n.overflow&&(d.overflow="hidden",b.support.shrinkWrapBlocks||f.always(function(){d.overflow=n.overflow[0],d.overflowX=n.overflow[1],d.overflowY=n.overflow[2]}));for(i in t)if(a=t[i],Vn.exec(a)){if(delete t[i],u=u||"toggle"===a,a===(m?"hide":"show"))continue;g.push(i)}if(o=g.length){s=b._data(e,"fxshow")||b._data(e,"fxshow",{}),"hidden"in s&&(m=s.hidden),u&&(s.hidden=!m),m?b(e).show():f.done(function(){b(e).hide()}),f.done(function(){var t;b._removeData(e,"fxshow");for(t in h)b.style(e,t,h[t])});for(i=0;o>i;i++)r=g[i],l=f.createTween(r,m?s[r]:0),h[r]=s[r]||b.style(e,r),r in s||(s[r]=l.start,m&&(l.end=l.start,l.start="width"===r||"height"===r?1:0))}}function rr(e,t,n,r,i){return new rr.prototype.init(e,t,n,r,i)}b.Tween=rr,rr.prototype={constructor:rr,init:function(e,t,n,r,i,o){this.elem=e,this.prop=n,this.easing=i||"swing",this.options=t,this.start=this.now=this.cur(),this.end=r,this.unit=o||(b.cssNumber[n]?"":"px")},cur:function(){var e=rr.propHooks[this.prop];return e&&e.get?e.get(this):rr.propHooks._default.get(this)},run:function(e){var t,n=rr.propHooks[this.prop];return this.pos=t=this.options.duration?b.easing[this.easing](e,this.options.duration*e,0,1,this.options.duration):e,this.now=(this.end-this.start)*t+this.start,this.options.step&&this.options.step.call(this.elem,this.now,this),n&&n.set?n.set(this):rr.propHooks._default.set(this),this}},rr.prototype.init.prototype=rr.prototype,rr.propHooks={_default:{get:function(e){var t;return null==e.elem[e.prop]||e.elem.style&&null!=e.elem.style[e.prop]?(t=b.css(e.elem,e.prop,""),t&&"auto"!==t?t:0):e.elem[e.prop]},set:function(e){b.fx.step[e.prop]?b.fx.step[e.prop](e):e.elem.style&&(null!=e.elem.style[b.cssProps[e.prop]]||b.cssHooks[e.prop])?b.style(e.elem,e.prop,e.now+e.unit):e.elem[e.prop]=e.now}}},rr.propHooks.scrollTop=rr.propHooks.scrollLeft={set:function(e){e.elem.nodeType&&e.elem.parentNode&&(e.elem[e.prop]=e.now)}},b.each(["toggle","show","hide"],function(e,t){var n=b.fn[t];b.fn[t]=function(e,r,i){return null==e||"boolean"==typeof e?n.apply(this,arguments):this.animate(ir(t,!0),e,r,i)}}),b.fn.extend({fadeTo:function(e,t,n,r){return this.filter(nn).css("opacity",0).show().end().animate({opacity:t},e,n,r)},animate:function(e,t,n,r){var i=b.isEmptyObject(e),o=b.speed(t,n,r),a=function(){var t=er(this,b.extend({},e),o);a.finish=function(){t.stop(!0)},(i||b._data(this,"finish"))&&t.stop(!0)};return a.finish=a,i||o.queue===!1?this.each(a):this.queue(o.queue,a)},stop:function(e,n,r){var i=function(e){var t=e.stop;delete e.stop,t(r)};return"string"!=typeof e&&(r=n,n=e,e=t),n&&e!==!1&&this.queue(e||"fx",[]),this.each(function(){var t=!0,n=null!=e&&e+"queueHooks",o=b.timers,a=b._data(this);if(n)a[n]&&a[n].stop&&i(a[n]);else for(n in a)a[n]&&a[n].stop&&Jn.test(n)&&i(a[n]);for(n=o.length;n--;)o[n].elem!==this||null!=e&&o[n].queue!==e||(o[n].anim.stop(r),t=!1,o.splice(n,1));(t||!r)&&b.dequeue(this,e)})},finish:function(e){return e!==!1&&(e=e||"fx"),this.each(function(){var t,n=b._data(this),r=n[e+"queue"],i=n[e+"queueHooks"],o=b.timers,a=r?r.length:0;for(n.finish=!0,b.queue(this,e,[]),i&&i.cur&&i.cur.finish&&i.cur.finish.call(this),t=o.length;t--;)o[t].elem===this&&o[t].queue===e&&(o[t].anim.stop(!0),o.splice(t,1));for(t=0;a>t;t++)r[t]&&r[t].finish&&r[t].finish.call(this);delete n.finish})}});function ir(e,t){var n,r={height:e},i=0;for(t=t?1:0;4>i;i+=2-t)n=Zt[i],r["margin"+n]=r["padding"+n]=e;return t&&(r.opacity=r.width=e),r}b.each({slideDown:ir("show"),slideUp:ir("hide"),slideToggle:ir("toggle"),fadeIn:{opacity:"show"},fadeOut:{opacity:"hide"},fadeToggle:{opacity:"toggle"}},function(e,t){b.fn[e]=function(e,n,r){return this.animate(t,e,n,r)}}),b.speed=function(e,t,n){var r=e&&"object"==typeof e?b.extend({},e):{complete:n||!n&&t||b.isFunction(e)&&e,duration:e,easing:n&&t||t&&!b.isFunction(t)&&t};return r.duration=b.fx.off?0:"number"==typeof r.duration?r.duration:r.duration in b.fx.speeds?b.fx.speeds[r.duration]:b.fx.speeds._default,(null==r.queue||r.queue===!0)&&(r.queue="fx"),r.old=r.complete,r.complete=function(){b.isFunction(r.old)&&r.old.call(this),r.queue&&b.dequeue(this,r.queue)},r},b.easing={linear:function(e){return e},swing:function(e){return.5-Math.cos(e*Math.PI)/2}},b.timers=[],b.fx=rr.prototype.init,b.fx.tick=function(){var e,n=b.timers,r=0;for(Xn=b.now();n.length>r;r++)e=n[r],e()||n[r]!==e||n.splice(r--,1);n.length||b.fx.stop(),Xn=t},b.fx.timer=function(e){e()&&b.timers.push(e)&&b.fx.start()},b.fx.interval=13,b.fx.start=function(){Un||(Un=setInterval(b.fx.tick,b.fx.interval))},b.fx.stop=function(){clearInterval(Un),Un=null},b.fx.speeds={slow:600,fast:200,_default:400},b.fx.step={},b.expr&&b.expr.filters&&(b.expr.filters.animated=function(e){return b.grep(b.timers,function(t){return e===t.elem}).length}),b.fn.offset=function(e){if(arguments.length)return e===t?this:this.each(function(t){b.offset.setOffset(this,e,t)});var n,r,o={top:0,left:0},a=this[0],s=a&&a.ownerDocument;if(s)return n=s.documentElement,b.contains(n,a)?(typeof a.getBoundingClientRect!==i&&(o=a.getBoundingClientRect()),r=or(s),{top:o.top+(r.pageYOffset||n.scrollTop)-(n.clientTop||0),left:o.left+(r.pageXOffset||n.scrollLeft)-(n.clientLeft||0)}):o},b.offset={setOffset:function(e,t,n){var r=b.css(e,"position");"static"===r&&(e.style.position="relative");var i=b(e),o=i.offset(),a=b.css(e,"top"),s=b.css(e,"left"),u=("absolute"===r||"fixed"===r)&&b.inArray("auto",[a,s])>-1,l={},c={},p,f;u?(c=i.position(),p=c.top,f=c.left):(p=parseFloat(a)||0,f=parseFloat(s)||0),b.isFunction(t)&&(t=t.call(e,n,o)),null!=t.top&&(l.top=t.top-o.top+p),null!=t.left&&(l.left=t.left-o.left+f),"using"in t?t.using.call(e,l):i.css(l)}},b.fn.extend({position:function(){if(this[0]){var e,t,n={top:0,left:0},r=this[0];return"fixed"===b.css(r,"position")?t=r.getBoundingClientRect():(e=this.offsetParent(),t=this.offset(),b.nodeName(e[0],"html")||(n=e.offset()),n.top+=b.css(e[0],"borderTopWidth",!0),n.left+=b.css(e[0],"borderLeftWidth",!0)),{top:t.top-n.top-b.css(r,"marginTop",!0),left:t.left-n.left-b.css(r,"marginLeft",!0)}}},offsetParent:function(){return this.map(function(){var e=this.offsetParent||o.documentElement;while(e&&!b.nodeName(e,"html")&&"static"===b.css(e,"position"))e=e.offsetParent;return e||o.documentElement})}}),b.each({scrollLeft:"pageXOffset",scrollTop:"pageYOffset"},function(e,n){var r=/Y/.test(n);b.fn[e]=function(i){return b.access(this,function(e,i,o){var a=or(e);return o===t?a?n in a?a[n]:a.document.documentElement[i]:e[i]:(a?a.scrollTo(r?b(a).scrollLeft():o,r?o:b(a).scrollTop()):e[i]=o,t)},e,i,arguments.length,null)}});function or(e){return b.isWindow(e)?e:9===e.nodeType?e.defaultView||e.parentWindow:!1}b.each({Height:"height",Width:"width"},function(e,n){b.each({padding:"inner"+e,content:n,"":"outer"+e},function(r,i){b.fn[i]=function(i,o){var a=arguments.length&&(r||"boolean"!=typeof i),s=r||(i===!0||o===!0?"margin":"border");return b.access(this,function(n,r,i){var o;return b.isWindow(n)?n.document.documentElement["client"+e]:9===n.nodeType?(o=n.documentElement,Math.max(n.body["scroll"+e],o["scroll"+e],n.body["offset"+e],o["offset"+e],o["client"+e])):i===t?b.css(n,r,s):b.style(n,r,i,s)},n,a?i:t,a,null)}})}),e.jQuery=e.$=b,"function"==typeof define&&define.amd&&define.amd.jQuery&&define("jquery",[],function(){return b})})(window);
diff --git a/asterix-app/src/main/resources/webui/static/js/smoothie.js b/asterix-app/src/main/resources/webui/static/js/smoothie.js
new file mode 100644
index 0000000..60c6624
--- /dev/null
+++ b/asterix-app/src/main/resources/webui/static/js/smoothie.js
@@ -0,0 +1,558 @@
+
+;(function(exports) {
+
+ var Util = {
+ extend: function() {
+ arguments[0] = arguments[0] || {};
+ for (var i = 1; i < arguments.length; i++)
+ {
+ for (var key in arguments[i])
+ {
+ if (arguments[i].hasOwnProperty(key))
+ {
+ if (typeof(arguments[i][key]) === 'object') {
+ if (arguments[i][key] instanceof Array) {
+ arguments[0][key] = arguments[i][key];
+ } else {
+ arguments[0][key] = Util.extend(arguments[0][key], arguments[i][key]);
+ }
+ } else {
+ arguments[0][key] = arguments[i][key];
+ }
+ }
+ }
+ }
+ return arguments[0];
+ }
+ };
+
+ /**
+ * Initialises a new <code>TimeSeries</code> with optional data options.
+ *
+ * Options are of the form (defaults shown):
+ *
+ * <pre>
+ * {
+ * resetBounds: true,
+ * resetBoundsInterval: 3000
+ * }
+ * </pre>
+ *
+ * Presentation options for TimeSeries are specified as an argument to <code>SmoothieChart.addTimeSeries</code>.
+ *
+ * @constructor
+ */
+ function TimeSeries(options) {
+ this.options = Util.extend({}, TimeSeries.defaultOptions, options);
+ this.data = [];
+ this.maxValue = Number.NaN;
+ this.minValue = Number.NaN;
+ }
+
+ TimeSeries.defaultOptions = {
+ resetBoundsInterval: 3000,
+ resetBounds: false
+ };
+
+ /**
+ * Recalculate the min/max values for this <code>TimeSeries</code> object.
+ *
+ * This causes the graph to scale itself in the y-axis.
+ */
+ TimeSeries.prototype.resetBounds = function() {
+ if (this.data.length) {
+
+ this.maxValue = this.data[0][1];
+ this.minValue = this.data[0][1];
+ for (var i = 1; i < this.data.length; i++) {
+ var value = this.data[i][1];
+ if (value > this.maxValue) {
+ this.maxValue = value;
+ }
+ if (value < this.minValue) {
+ this.minValue = value;
+ }
+ }
+ } else {
+
+ this.maxValue = Number.NaN;
+ this.minValue = Number.NaN;
+ }
+ };
+
+ /**
+ * Adds a new data point to the <code>TimeSeries</code>, preserving chronological order.
+ *
+ * @param timestamp the position, in time, of this data point
+ * @param value the value of this data point
+ * @param sumRepeatedTimeStampValues if <code>timestamp</code> has an exact match in the series, this flag controls
+ * whether it is replaced, or the values summed (defaults to false.)
+ */
+ TimeSeries.prototype.append = function(timestamp, value, sumRepeatedTimeStampValues) {
+
+ var i = this.data.length - 1;
+ while (i > 0 && this.data[i][0] > timestamp) {
+ i--;
+ }
+
+ if (this.data.length > 0 && this.data[i][0] === timestamp) {
+
+ if (sumRepeatedTimeStampValues) {
+
+ this.data[i][1] += value;
+ value = this.data[i][1];
+ } else {
+
+ this.data[i][1] = value;
+ }
+ } else if (i < this.data.length - 1) {
+
+ this.data.splice(i + 1, 0, [timestamp, value]);
+ } else {
+
+ this.data.push([timestamp, value]);
+ }
+
+ this.maxValue = isNaN(this.maxValue) ? value : Math.max(this.maxValue, value);
+ this.minValue = isNaN(this.minValue) ? value : Math.min(this.minValue, value);
+ };
+
+ TimeSeries.prototype.dropOldData = function(oldestValidTime, maxDataSetLength) {
+
+
+ var removeCount = 0;
+ while (this.data.length - removeCount >= maxDataSetLength && this.data[removeCount + 1][0] < oldestValidTime) {
+ removeCount++;
+ }
+ if (removeCount !== 0) {
+ this.data.splice(0, removeCount);
+ }
+ };
+
+ /**
+ * Initialises a new <code>SmoothieChart</code>.
+ *
+ * Options are optional, and should be of the form below. Just specify the values you
+ * need and the rest will be given sensible defaults as shown:
+ *
+ * <pre>
+ * {
+ * minValue: undefined,
+ * maxValue: undefined,
+ * maxValueScale: 1,
+ * yRangeFunction: undefined,
+ * scaleSmoothing: 0.125,
+ * millisPerPixel: 20,
+ * maxDataSetLength: 2,
+ * interpolation: 'bezier'
+ * timestampFormatter: null,
+ * horizontalLines: [],
+ * grid:
+ * {
+ * fillStyle: '#000000',
+ * lineWidth: 1,
+ * strokeStyle: '#777777',
+ * millisPerLine: 1000,
+ * sharpLines: false,
+ * verticalSections: 2,
+ * borderVisible: true
+ * },
+ * labels
+ * {
+ * disabled: false,
+ * fillStyle: '#ffffff',
+ * fontSize: 15,
+ * fontFamily: 'sans-serif',
+ * precision: 2
+ * },
+ * }
+ * </pre>
+ *
+ * @constructor
+ */
+ function SmoothieChart(options) {
+ this.options = Util.extend({}, SmoothieChart.defaultChartOptions, options);
+ this.seriesSet = [];
+ this.currentValueRange = 1;
+ this.currentVisMinValue = 0;
+ }
+
+ SmoothieChart.defaultChartOptions = {
+ millisPerPixel: 20,
+ maxValueScale: 1,
+ interpolation: 'bezier',
+ scaleSmoothing: 0.125,
+ maxDataSetLength: 2,
+ grid: {
+ fillStyle: '#000000',
+ strokeStyle: '#777777',
+ lineWidth: 1,
+ sharpLines: false,
+ millisPerLine: 1000,
+ verticalSections: 2,
+ borderVisible: true
+ },
+ labels: {
+ fillStyle: '#ffffff',
+ disabled: false,
+ fontSize: 10,
+ fontFamily: 'monospace',
+ precision: 2
+ },
+ horizontalLines: []
+ };
+
+ SmoothieChart.AnimateCompatibility = (function() {
+ var lastTime = 0,
+ requestAnimationFrame = function(callback, element) {
+ var requestAnimationFrame =
+ window.requestAnimationFrame ||
+ window.webkitRequestAnimationFrame ||
+ window.mozRequestAnimationFrame ||
+ window.oRequestAnimationFrame ||
+ window.msRequestAnimationFrame ||
+ function(callback) {
+ var currTime = new Date().getTime(),
+ timeToCall = Math.max(0, 16 - (currTime - lastTime)),
+ id = window.setTimeout(function() {
+ callback(currTime + timeToCall);
+ }, timeToCall);
+ lastTime = currTime + timeToCall;
+ return id;
+ };
+ return requestAnimationFrame.call(window, callback, element);
+ },
+ cancelAnimationFrame = function(id) {
+ var cancelAnimationFrame =
+ window.cancelAnimationFrame ||
+ function(id) {
+ clearTimeout(id);
+ };
+ return cancelAnimationFrame.call(window, id);
+ };
+
+ return {
+ requestAnimationFrame: requestAnimationFrame,
+ cancelAnimationFrame: cancelAnimationFrame
+ };
+ })();
+
+ SmoothieChart.defaultSeriesPresentationOptions = {
+ lineWidth: 1,
+ strokeStyle: '#ffffff'
+ };
+
+ /**
+ * Adds a <code>TimeSeries</code> to this chart, with optional presentation options.
+ *
+ * Presentation options should be of the form (defaults shown):
+ *
+ * <pre>
+ * {
+ * lineWidth: 1,
+ * strokeStyle: '#ffffff',
+ * fillStyle: undefined
+ * }
+ * </pre>
+ */
+ SmoothieChart.prototype.addTimeSeries = function(timeSeries, options) {
+ this.seriesSet.push({timeSeries: timeSeries, options: Util.extend({}, SmoothieChart.defaultSeriesPresentationOptions, options)});
+ if (timeSeries.options.resetBounds && timeSeries.options.resetBoundsInterval > 0) {
+ timeSeries.resetBoundsTimerId = setInterval(
+ function() {
+ timeSeries.resetBounds();
+ },
+ timeSeries.options.resetBoundsInterval
+ );
+ }
+ };
+
+ /**
+ * Removes the specified <code>TimeSeries</code> from the chart.
+ */
+ SmoothieChart.prototype.removeTimeSeries = function(timeSeries) {
+ var numSeries = this.seriesSet.length;
+ for (var i = 0; i < numSeries; i++) {
+ if (this.seriesSet[i].timeSeries === timeSeries) {
+ this.seriesSet.splice(i, 1);
+ break;
+ }
+ }
+ if (timeSeries.resetBoundsTimerId) {
+ clearInterval(timeSeries.resetBoundsTimerId);
+ }
+ };
+
+ /**
+ * Instructs the <code>SmoothieChart</code> to start rendering to the provided canvas, with specified delay.
+ *
+ * @param canvas the target canvas element
+ * @param delayMillis an amount of time to wait before a data point is shown. This can prevent the end of the series
+ * from appearing on screen, with new values flashing into view, at the expense of some latency.
+ */
+ SmoothieChart.prototype.streamTo = function(canvas, delayMillis) {
+ this.canvas = canvas;
+ this.delay = delayMillis;
+ this.start();
+ };
+
+ /**
+ * Starts the animation of this chart.
+ */
+ SmoothieChart.prototype.start = function() {
+ if (this.frame) {
+ return;
+ }
+
+ var animate = function() {
+ this.frame = SmoothieChart.AnimateCompatibility.requestAnimationFrame(function() {
+ this.render();
+ animate();
+ }.bind(this));
+ }.bind(this);
+
+ animate();
+ };
+
+ /**
+ * Stops the animation of this chart.
+ */
+ SmoothieChart.prototype.stop = function() {
+ if (this.frame) {
+ SmoothieChart.AnimateCompatibility.cancelAnimationFrame(this.frame);
+ delete this.frame;
+ }
+ };
+
+ SmoothieChart.prototype.updateValueRange = function() {
+ var chartOptions = this.options,
+ chartMaxValue = Number.NaN,
+ chartMinValue = Number.NaN;
+
+ for (var d = 0; d < this.seriesSet.length; d++) {
+ var timeSeries = this.seriesSet[d].timeSeries;
+ if (!isNaN(timeSeries.maxValue)) {
+ chartMaxValue = !isNaN(chartMaxValue) ? Math.max(chartMaxValue, timeSeries.maxValue) : timeSeries.maxValue;
+ }
+
+ if (!isNaN(timeSeries.minValue)) {
+ chartMinValue = !isNaN(chartMinValue) ? Math.min(chartMinValue, timeSeries.minValue) : timeSeries.minValue;
+ }
+ }
+
+ if (chartOptions.maxValue != null) {
+ chartMaxValue = chartOptions.maxValue;
+ } else {
+ chartMaxValue *= chartOptions.maxValueScale;
+ }
+
+ if (chartOptions.minValue != null) {
+ chartMinValue = chartOptions.minValue;
+ }
+
+ if (this.options.yRangeFunction) {
+ var range = this.options.yRangeFunction({min: chartMinValue, max: chartMaxValue});
+ chartMinValue = range.min;
+ chartMaxValue = range.max;
+ }
+
+ if (!isNaN(chartMaxValue) && !isNaN(chartMinValue)) {
+ var targetValueRange = chartMaxValue - chartMinValue;
+ this.currentValueRange += chartOptions.scaleSmoothing * (targetValueRange - this.currentValueRange);
+ this.currentVisMinValue += chartOptions.scaleSmoothing * (chartMinValue - this.currentVisMinValue);
+ }
+
+ this.valueRange = { min: chartMinValue, max: chartMaxValue };
+ };
+
+ SmoothieChart.prototype.render = function(canvas, time) {
+ canvas = canvas || this.canvas;
+ time = time || new Date().getTime() - (this.delay || 0);
+
+
+ time -= time % this.options.millisPerPixel;
+
+ var context = canvas.getContext('2d'),
+ chartOptions = this.options,
+ dimensions = { top: 0, left: 0, width: canvas.clientWidth, height: canvas.clientHeight },
+ oldestValidTime = time - (dimensions.width * chartOptions.millisPerPixel),
+ valueToYPixel = function(value) {
+ var offset = value - this.currentVisMinValue;
+ return this.currentValueRange === 0
+ ? dimensions.height
+ : dimensions.height - (Math.round((offset / this.currentValueRange) * dimensions.height));
+ }.bind(this),
+ timeToXPixel = function(t) {
+ return Math.round(dimensions.width - ((time - t) / chartOptions.millisPerPixel));
+ };
+
+ this.updateValueRange();
+
+ context.font = chartOptions.labels.fontSize + 'px ' + chartOptions.labels.fontFamily;
+
+ context.save();
+
+ context.translate(dimensions.left, dimensions.top);
+
+ context.beginPath();
+ context.rect(0, 0, dimensions.width, dimensions.height);
+ context.clip();
+
+ context.save();
+ context.fillStyle = chartOptions.grid.fillStyle;
+ context.clearRect(0, 0, dimensions.width, dimensions.height);
+ context.fillRect(0, 0, dimensions.width, dimensions.height);
+ context.restore();
+
+ context.save();
+ context.lineWidth = chartOptions.grid.lineWidth;
+ context.strokeStyle = chartOptions.grid.strokeStyle;
+ if (chartOptions.grid.millisPerLine > 0) {
+ var textUntilX = dimensions.width - context.measureText(minValueString).width + 4;
+ for (var t = time - (time % chartOptions.grid.millisPerLine);
+ t >= oldestValidTime;
+ t -= chartOptions.grid.millisPerLine) {
+ var gx = timeToXPixel(t);
+ if (chartOptions.grid.sharpLines) {
+ gx -= 0.5;
+ }
+ context.beginPath();
+ context.moveTo(gx, 0);
+ context.lineTo(gx, dimensions.height);
+ context.stroke();
+ context.closePath();
+
+ if (chartOptions.timestampFormatter && gx < textUntilX) {
+
+ var tx = new Date(t),
+ ts = chartOptions.timestampFormatter(tx),
+ tsWidth = context.measureText(ts).width;
+ textUntilX = gx - tsWidth - 2;
+ context.fillStyle = chartOptions.labels.fillStyle;
+ context.fillText(ts, gx - tsWidth, dimensions.height - 2);
+ }
+ }
+ }
+
+ for (var v = 1; v < chartOptions.grid.verticalSections; v++) {
+ var gy = Math.round(v * dimensions.height / chartOptions.grid.verticalSections);
+ if (chartOptions.grid.sharpLines) {
+ gy -= 0.5;
+ }
+ context.beginPath();
+ context.moveTo(0, gy);
+ context.lineTo(dimensions.width, gy);
+ context.stroke();
+ context.closePath();
+ }
+ if (chartOptions.grid.borderVisible) {
+ context.beginPath();
+ context.strokeRect(0, 0, dimensions.width, dimensions.height);
+ context.closePath();
+ }
+ context.restore();
+
+ if (chartOptions.horizontalLines && chartOptions.horizontalLines.length) {
+ for (var hl = 0; hl < chartOptions.horizontalLines.length; hl++) {
+ var line = chartOptions.horizontalLines[hl],
+ hly = Math.round(valueToYPixel(line.value)) - 0.5;
+ context.strokeStyle = line.color || '#ffffff';
+ context.lineWidth = line.lineWidth || 1;
+ context.beginPath();
+ context.moveTo(0, hly);
+ context.lineTo(dimensions.width, hly);
+ context.stroke();
+ context.closePath();
+ }
+ }
+
+ for (var d = 0; d < this.seriesSet.length; d++) {
+ context.save();
+ var timeSeries = this.seriesSet[d].timeSeries,
+ dataSet = timeSeries.data,
+ seriesOptions = this.seriesSet[d].options;
+
+ timeSeries.dropOldData(oldestValidTime, chartOptions.maxDataSetLength);
+
+ context.lineWidth = seriesOptions.lineWidth;
+ context.strokeStyle = seriesOptions.strokeStyle;
+ context.beginPath();
+ var firstX = 0, lastX = 0, lastY = 0;
+ for (var i = 0; i < dataSet.length && dataSet.length !== 1; i++) {
+ var x = timeToXPixel(dataSet[i][0]),
+ y = valueToYPixel(dataSet[i][1]);
+
+ if (i === 0) {
+ firstX = x;
+ context.moveTo(x, y);
+ } else {
+ switch (chartOptions.interpolation) {
+ case "linear":
+ case "line": {
+ context.lineTo(x,y);
+ break;
+ }
+ case "bezier":
+ default: {
+
+
+
+
+
+
+
+
+
+ context.bezierCurveTo(
+ Math.round((lastX + x) / 2), lastY,
+ Math.round((lastX + x)) / 2, y,
+ x, y);
+ break;
+ }
+ }
+ }
+
+ lastX = x; lastY = y;
+ }
+
+ if (dataSet.length > 1) {
+ if (seriesOptions.fillStyle) {
+
+ context.lineTo(dimensions.width + seriesOptions.lineWidth + 1, lastY);
+ context.lineTo(dimensions.width + seriesOptions.lineWidth + 1, dimensions.height + seriesOptions.lineWidth + 1);
+ context.lineTo(firstX, dimensions.height + seriesOptions.lineWidth);
+ context.fillStyle = seriesOptions.fillStyle;
+ context.fill();
+ }
+
+ if (seriesOptions.strokeStyle && seriesOptions.strokeStyle !== 'none') {
+ context.stroke();
+ }
+ context.closePath();
+ }
+ context.restore();
+ }
+
+
+ if (!chartOptions.labels.disabled && !isNaN(this.valueRange.min) && !isNaN(this.valueRange.max)) {
+ var maxValueString = parseFloat(this.valueRange.max).toFixed(chartOptions.labels.precision),
+ minValueString = parseFloat(this.valueRange.min).toFixed(chartOptions.labels.precision);
+ context.fillStyle = chartOptions.labels.fillStyle;
+ context.fillText(maxValueString, dimensions.width - context.measureText(maxValueString).width - 2, chartOptions.labels.fontSize);
+ context.fillText(minValueString, dimensions.width - context.measureText(minValueString).width - 2, dimensions.height - 2);
+ }
+
+ context.restore();
+ };
+
+
+ SmoothieChart.timeFormatter = function(date) {
+ function pad2(number) { return (number < 10 ? '0' : '') + number }
+ return pad2(date.getHours()) + ':' + pad2(date.getMinutes()) + ':' + pad2(date.getSeconds());
+ };
+
+ exports.TimeSeries = TimeSeries;
+ exports.SmoothieChart = SmoothieChart;
+
+})(typeof exports === 'undefined' ? this : exports);
+
diff --git a/asterix-app/src/test/java/edu/uci/ics/asterix/test/metadata/MetadataTest.java b/asterix-app/src/test/java/edu/uci/ics/asterix/test/metadata/MetadataTest.java
index 91b75c5..6afc331 100644
--- a/asterix-app/src/test/java/edu/uci/ics/asterix/test/metadata/MetadataTest.java
+++ b/asterix-app/src/test/java/edu/uci/ics/asterix/test/metadata/MetadataTest.java
@@ -105,7 +105,9 @@
@Test
public void test() throws Exception {
+ // if (tcCtx.getTestCase().getCompilationUnit().get(0).getName().contains("meta09")) {
TestsUtils.executeTest(PATH_ACTUAL, tcCtx);
+ // }
}
}
\ No newline at end of file
diff --git a/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java b/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
index 36bf23c..21caf28 100644
--- a/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
+++ b/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
@@ -17,6 +17,8 @@
import java.io.File;
import java.util.ArrayList;
import java.util.Collection;
+import java.util.logging.Level;
+import java.util.logging.Logger;
import org.apache.commons.io.FileUtils;
import org.junit.AfterClass;
@@ -40,6 +42,9 @@
*/
@RunWith(Parameterized.class)
public class ExecutionTest {
+
+ private static final Logger LOGGER = Logger.getLogger(ExecutionTest.class.getName());
+
private static final String PATH_ACTUAL = "rttest/";
private static final String PATH_BASE = "src/test/resources/runtimets/";
@@ -50,6 +55,10 @@
@BeforeClass
public static void setUp() throws Exception {
+ System.out.println("Starting setup");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Starting setup");
+ }
System.setProperty(GlobalConfig.CONFIG_FILE_PROPERTY, TEST_CONFIG_FILE_NAME);
System.setProperty(GlobalConfig.WEB_SERVER_PORT_PROPERTY, "19002");
File outdir = new File(PATH_ACTUAL);
@@ -60,16 +69,22 @@
deleteTransactionLogs();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("initializing pseudo cluster");
+ }
AsterixHyracksIntegrationUtil.init();
- // TODO: Uncomment when hadoop version is upgraded and adapters are
- // ported.
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("initializing HDFS");
+ }
+
HDFSCluster.getInstance().setup();
// Set the node resolver to be the identity resolver that expects node names
// to be node controller ids; a valid assumption in test environment.
System.setProperty(FileSystemBasedAdapter.NODE_RESOLVER_FACTORY_PROPERTY,
IdentitiyResolverFactory.class.getName());
+
}
@AfterClass
diff --git a/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/HDFSCluster.java b/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/HDFSCluster.java
index d6a813c..2318a04 100644
--- a/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/HDFSCluster.java
+++ b/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/HDFSCluster.java
@@ -88,8 +88,10 @@
}
public void cleanup() throws Exception {
- dfsCluster.shutdown();
- cleanupLocal();
+ if (dfsCluster != null) {
+ dfsCluster.shutdown();
+ cleanupLocal();
+ }
}
public static void main(String[] args) throws Exception {
diff --git a/asterix-app/src/test/resources/hadoop/conf/core-site.xml b/asterix-app/src/test/resources/hadoop/conf/core-site.xml
index 5b1023c..433325a 100644
--- a/asterix-app/src/test/resources/hadoop/conf/core-site.xml
+++ b/asterix-app/src/test/resources/hadoop/conf/core-site.xml
@@ -21,7 +21,7 @@
<property>
<name>fs.default.name</name>
- <value>hdfs://127.0.0.1:31888</value>
+ <value>hdfs://192.168.0.103:31888</value>
</property>
<property>
<name>hadoop.tmp.dir</name>
diff --git a/asterix-app/src/test/resources/logging.properties b/asterix-app/src/test/resources/logging.properties
index cf1457f..c904be4 100644
--- a/asterix-app/src/test/resources/logging.properties
+++ b/asterix-app/src/test/resources/logging.properties
@@ -79,3 +79,5 @@
#edu.uci.ics.asterix.level = FINE
#edu.uci.ics.hyracks.algebricks.level = FINE
#edu.uci.ics.hyracks.level = INFO
+edu.uci.ics.asterix.test = INFO
+edu.uci.ics.asterix.installer.test = INFO
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2.adm
index 7c82b18..7a6c4e0 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Tue Jan 29 19:11:26 PST 2013" }
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Tue Jul 16 23:01:41 PDT 2013", "DatasetId": 106, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2/issue_251_dataset_hint_2.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2/issue_251_dataset_hint_2.1.adm
index 0fb70db..e29398c 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2/issue_251_dataset_hint_2.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2/issue_251_dataset_hint_2.1.adm
@@ -1 +1,5 @@
-{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Mon Sep 23 00:01:03 PDT 2013", "DatasetId": 106, "PendingOp": 0 }
\ No newline at end of file
+<<<<<<< HEAD
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Tue Jul 16 23:01:41 PDT 2013", "DatasetId": 106, "PendingOp": 0 }
+=======
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Mon Sep 23 00:01:03 PDT 2013", "DatasetId": 106, "PendingOp": 0 }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3.adm
index f931b40..991fa32 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Tue Jan 29 19:00:38 PST 2013" }
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Tue Jul 16 22:49:42 PDT 2013", "DatasetId": 107, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3/issue_251_dataset_hint_3.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3/issue_251_dataset_hint_3.1.adm
index 9ec7ae7..e5e8615 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3/issue_251_dataset_hint_3.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3/issue_251_dataset_hint_3.1.adm
@@ -1 +1,5 @@
-{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Mon Sep 23 00:04:09 PDT 2013", "DatasetId": 107, "PendingOp": 0 }
\ No newline at end of file
+<<<<<<< HEAD
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Tue Jul 16 22:49:42 PDT 2013", "DatasetId": 107, "PendingOp": 0 }
+=======
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Mon Sep 23 00:04:09 PDT 2013", "DatasetId": 107, "PendingOp": 0 }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4.adm
index efd3a7e..7aca625 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:59:57 PST 2013" }
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:00:46 PDT 2013", "DatasetId": 108, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4/issue_251_dataset_hint_4.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4/issue_251_dataset_hint_4.1.adm
index 98f9d7d..23b465c 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4/issue_251_dataset_hint_4.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4/issue_251_dataset_hint_4.1.adm
@@ -1 +1,5 @@
-{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:25:30 PDT 2013", "DatasetId": 108, "PendingOp": 0 }
\ No newline at end of file
+<<<<<<< HEAD
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:00:46 PDT 2013", "DatasetId": 108, "PendingOp": 0 }
+=======
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:25:30 PDT 2013", "DatasetId": 108, "PendingOp": 0 }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta02.adm b/asterix-app/src/test/resources/metadata/results/basic/meta02.adm
index 394af5c..027f12b 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta02.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta02.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatasetName": "dst01", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:54:34 PST 2013" }
+{ "DataverseName": "testdv", "DatasetName": "dst01", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:35:01 PDT 2013", "DatasetId": 101, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta02/meta02.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta02/meta02.1.adm
index 8fb7bc2..c034bb5 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta02/meta02.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta02/meta02.1.adm
@@ -1 +1,5 @@
-{ "DataverseName": "testdv", "DatasetName": "dst01", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:06:09 PDT 2013", "DatasetId": 101, "PendingOp": 0 }
\ No newline at end of file
+<<<<<<< HEAD
+{ "DataverseName": "testdv", "DatasetName": "dst01", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:35:01 PDT 2013", "DatasetId": 101, "PendingOp": 0 }
+=======
+{ "DataverseName": "testdv", "DatasetName": "dst01", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:06:09 PDT 2013", "DatasetId": 101, "PendingOp": 0 }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta09.adm b/asterix-app/src/test/resources/metadata/results/basic/meta09.adm
index 9bcb2a4..d0ec344 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta09.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta09.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:55:25 PST 2013" }
+{ "DataverseName": "test", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:28:17 PDT 2013", "DatasetId": 101, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta09/meta09.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta09/meta09.1.adm
index 371ee07..64b3391 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta09/meta09.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta09/meta09.1.adm
@@ -1 +1,5 @@
-{ "DataverseName": "test", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:04:08 PDT 2013", "DatasetId": 103, "PendingOp": 0 }
\ No newline at end of file
+<<<<<<< HEAD
+{ "DataverseName": "test", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:28:17 PDT 2013", "DatasetId": 101, "PendingOp": 0 }
+=======
+{ "DataverseName": "test", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:04:08 PDT 2013", "DatasetId": 103, "PendingOp": 0 }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta15.adm b/asterix-app/src/test/resources/metadata/results/basic/meta15.adm
index 4414ed0..43cf9ec 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta15.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta15.adm
@@ -1,6 +1,10 @@
-{ "DataverseName": "Metadata", "Name": "cnn_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.CNNFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "hdfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "hive", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "localfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.NCFileSystemAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "pull_twitter", "Classname": "edu.uci.ics.asterix.external.adapter.factory.PullBasedTwitterAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "rss_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.RSSFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
+{ "DataverseName": "Metadata", "Name": "cnn_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.CNNFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "file_feed", "Classname": "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "generic_socket_feed", "Classname": "edu.uci.ics.asterix.tools.external.data.GenericSocketFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "hdfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "hive", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "localfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.NCFileSystemAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "pull_twitter", "Classname": "edu.uci.ics.asterix.external.adapter.factory.PullBasedTwitterAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "rss_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.RSSFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "synthetic_twitter_feed", "Classname": "edu.uci.ics.asterix.tools.external.data.SyntheticTwitterFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "twitter_firehose", "Classname": "edu.uci.ics.asterix.tools.external.data.TwitterFirehoseFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta15/meta15.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta15/meta15.1.adm
index 4414ed0..43cf9ec 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta15/meta15.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta15/meta15.1.adm
@@ -1,6 +1,10 @@
-{ "DataverseName": "Metadata", "Name": "cnn_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.CNNFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "hdfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "hive", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "localfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.NCFileSystemAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "pull_twitter", "Classname": "edu.uci.ics.asterix.external.adapter.factory.PullBasedTwitterAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
-{ "DataverseName": "Metadata", "Name": "rss_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.RSSFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Sun Nov 25 20:55:22 PST 2012" }
+{ "DataverseName": "Metadata", "Name": "cnn_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.CNNFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "file_feed", "Classname": "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "generic_socket_feed", "Classname": "edu.uci.ics.asterix.tools.external.data.GenericSocketFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "hdfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "hive", "Classname": "edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "localfs", "Classname": "edu.uci.ics.asterix.external.adapter.factory.NCFileSystemAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "pull_twitter", "Classname": "edu.uci.ics.asterix.external.adapter.factory.PullBasedTwitterAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "rss_feed", "Classname": "edu.uci.ics.asterix.external.adapter.factory.RSSFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "synthetic_twitter_feed", "Classname": "edu.uci.ics.asterix.tools.external.data.SyntheticTwitterFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
+{ "DataverseName": "Metadata", "Name": "twitter_firehose", "Classname": "edu.uci.ics.asterix.tools.external.data.TwitterFirehoseFeedAdapterFactory", "Type": "INTERNAL", "Timestamp": "Tue Jul 16 22:38:45 PDT 2013" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta16.adm b/asterix-app/src/test/resources/metadata/results/basic/meta16.adm
index b56fe7c..aecf8a8 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta16.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta16.adm
@@ -1,8 +1,12 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName" ], "PrimaryKey": [ "DataverseName", "FeedName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 10, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 11, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 12, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 7, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 4, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 5, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
index 414d951..3788cc5 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
@@ -1,3 +1,17 @@
+<<<<<<< HEAD
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName" ], "PrimaryKey": [ "DataverseName", "FeedName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 10, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 11, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 12, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 7, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 4, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 5, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
+=======
{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "CompactionPolicy" ], "PrimaryKey": [ "DataverseName", "CompactionPolicy" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
@@ -6,4 +20,5 @@
{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "DatasetId": 7, "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "DatasetId": 4, "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "DatasetId": 5, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta17.adm b/asterix-app/src/test/resources/metadata/results/basic/meta17.adm
index 2de89a6..686aea5 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta17.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta17.adm
@@ -1,60 +1,66 @@
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "FeedDetails", "FieldType": "Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:30 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:30 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:30 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Status", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedActivityRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "ActivityId", "FieldType": "int32" }, { "FieldName": "ActivityType", "FieldType": "string" }, { "FieldName": "Details", "FieldType": "Field_Details_in_FeedActivityRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Description", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_FeedPolicyRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "AdaptorName", "FieldType": "string" }, { "FieldName": "AdaptorConfiguration", "FieldType": "Field_AdaptorConfiguration_in_FeedRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_FeedRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Details_in_FeedActivityRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_FeedPolicyRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
index 1995946..c749dcc 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
@@ -1,3 +1,71 @@
+<<<<<<< HEAD
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedActivityRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "ActivityId", "FieldType": "int32" }, { "FieldName": "ActivityType", "FieldType": "string" }, { "FieldName": "Details", "FieldType": "Field_Details_in_FeedActivityRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Description", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_FeedPolicyRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "AdaptorName", "FieldType": "string" }, { "FieldName": "AdaptorConfiguration", "FieldType": "Field_AdaptorConfiguration_in_FeedRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_FeedRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Details_in_FeedActivityRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_FeedPolicyRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013" }
+=======
{ "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013" }
{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "FeedDetails", "FieldType": "Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013" }
{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013" }
@@ -64,4 +132,5 @@
{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013" }
{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013" }
{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013" }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013" }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta19.adm b/asterix-app/src/test/resources/metadata/results/basic/meta19.adm
index 607bfd1..ee088a1 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta19.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta19.adm
@@ -1,11 +1,15 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "IndexName": "FeedActivity", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "PolicyName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm
index 7fdcb8d..f744e61 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm
@@ -1,3 +1,20 @@
+<<<<<<< HEAD
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "IndexName": "FeedActivity", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "PolicyName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:46:42 PDT 2013", "PendingOp": 0 }
+=======
{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "IndexName": "CompactionPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "CompactionPolicy" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "PendingOp": 0 }
@@ -9,4 +26,5 @@
{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:04:06 PDT 2013", "PendingOp": 0 }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset.adm
index 8abc339..2de7431 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset.adm
@@ -1,8 +1,12 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName" ], "PrimaryKey": [ "DataverseName", "FeedName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 10, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 11, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 12, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 7, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 4, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 5, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
index edf8f9d..ff87550 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
@@ -1,3 +1,17 @@
+<<<<<<< HEAD
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName" ], "PrimaryKey": [ "DataverseName", "FeedName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 10, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 11, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 12, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 7, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 4, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 5, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
+=======
{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "CompactionPolicy" ], "PrimaryKey": [ "DataverseName", "CompactionPolicy" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:06:07 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:06:07 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:06:07 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
@@ -6,4 +20,5 @@
{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:06:07 PDT 2013", "DatasetId": 7, "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:06:07 PDT 2013", "DatasetId": 4, "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:06:07 PDT 2013", "DatasetId": 5, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:06:07 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Mon Sep 23 00:06:07 PDT 2013", "DatasetId": 6, "PendingOp": 0 }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype.adm
index b351cfb..ea5e89b 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype.adm
@@ -1,56 +1,66 @@
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "FeedDetails", "FieldType": "Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Function", "FieldType": "string" }, { "FieldName": "Status", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedActivityRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "ActivityId", "FieldType": "int32" }, { "FieldName": "ActivityType", "FieldType": "string" }, { "FieldName": "Details", "FieldType": "Field_Details_in_FeedActivityRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Description", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_FeedPolicyRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "AdaptorName", "FieldType": "string" }, { "FieldName": "AdaptorConfiguration", "FieldType": "Field_AdaptorConfiguration_in_FeedRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_FeedRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Details_in_FeedActivityRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_FeedPolicyRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
index 4c22058..b2f62a2 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
@@ -1,3 +1,71 @@
+<<<<<<< HEAD
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedActivityRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "ActivityId", "FieldType": "int32" }, { "FieldName": "ActivityType", "FieldType": "string" }, { "FieldName": "Details", "FieldType": "Field_Details_in_FeedActivityRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Description", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_FeedPolicyRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "AdaptorName", "FieldType": "string" }, { "FieldName": "AdaptorConfiguration", "FieldType": "Field_AdaptorConfiguration_in_FeedRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_FeedRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Details_in_FeedActivityRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_FeedPolicyRecordType_ItemType", "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013" }
+=======
{ "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013" }
{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "FeedDetails", "FieldType": "Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013" }
{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013" }
@@ -64,4 +132,5 @@
{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013" }
{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013" }
{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013" }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013" }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_index.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_index.adm
index 607bfd1..a4ee9d3 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_index.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_index.adm
@@ -1,11 +1,15 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "IndexName": "FeedActivity", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "PolicyName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
index 072bcde..b65906f 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
@@ -1,3 +1,20 @@
+<<<<<<< HEAD
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "IndexName": "FeedActivity", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "PolicyName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Tue Jul 16 22:49:39 PDT 2013", "PendingOp": 0 }
+=======
{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "IndexName": "CompactionPolicy", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "CompactionPolicy" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013", "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013", "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013", "PendingOp": 0 }
@@ -9,4 +26,5 @@
{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013", "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013", "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013", "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Mon Sep 23 00:25:26 PDT 2013", "PendingOp": 0 }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/nontagged/custord/local/all-scan.aql b/asterix-app/src/test/resources/nontagged/custord/local/all-scan.aql
index 1bd4b73..cd51b5e 100644
--- a/asterix-app/src/test/resources/nontagged/custord/local/all-scan.aql
+++ b/asterix-app/src/test/resources/nontagged/custord/local/all-scan.aql
@@ -52,4 +52,4 @@
let $c3 := int32("320")
let $c4 := int64("640")
return {"int8": $c1,"int16": $c2,"int32": $c3, "int8co": $o.int8co, "int64": $c4}
-*/
\ No newline at end of file
+*/
diff --git a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.2.update.aql
deleted file mode 100644
index e69de29..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.2.update.aql
+++ /dev/null
diff --git a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.1.ddl.aql
deleted file mode 100644
index e69de29..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.1.ddl.aql
+++ /dev/null
diff --git a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.2.update.aql
deleted file mode 100644
index e69de29..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.2.update.aql
+++ /dev/null
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-record-function/insert-record-function.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-record-function/insert-record-function.1.ddl.aql
new file mode 100644
index 0000000..1883c94
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-record-function/insert-record-function.1.ddl.aql
@@ -0,0 +1,24 @@
+/*
+ * Test case Name : insert-into-empty-dataset.aql
+ * Description : Check that we can insert into an empty dataset
+ * Expected Result : Success
+ * Date : May 2 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type LineIDType as closed {
+ l_orderkey: int32,
+ l_linenumber: int32,
+ l_suppkey: int32
+}
+
+create dataset LineID(LineIDType)
+ primary key l_orderkey, l_linenumber;
+
+create dataset LineID2(LineIDType)
+ primary key l_orderkey, l_linenumber;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-record-function/insert-record-function.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-record-function/insert-record-function.2.update.aql
new file mode 100644
index 0000000..e155837
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-record-function/insert-record-function.2.update.aql
@@ -0,0 +1,35 @@
+/*
+ * Test case Name : insert-into-empty-dataset.aql
+ * Description : Check that we can insert into an empty dataset
+ * Expected Result : Success
+ * Date : May 2 2012
+ */
+
+use dataverse test;
+
+insert into dataset LineID (
+let $x:=1
+let $y:=2
+let $z:=3
+return {
+ "l_orderkey": $x,
+ "l_linenumber": $y,
+ "l_suppkey": $z
+}
+);
+
+insert into dataset LineID (
+let $x:=2
+let $y:=3
+let $z:=4
+return {
+ "l_orderkey": $x,
+ "l_linenumber": $y,
+ "l_suppkey": $z
+}
+);
+
+insert into dataset LineID2 (
+ for $x in dataset LineID
+ return flow-record($x)
+);
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-record-function/insert-record-function.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-record-function/insert-record-function.3.query.aql
new file mode 100644
index 0000000..e4763e5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-record-function/insert-record-function.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Test case Name : insert-into-empty-dataset.aql
+ * Description : Check that we can insert into an empty dataset
+ * Expected Result : Success
+ * Date : May 2 2012
+ */
+
+use dataverse test;
+
+for $c in dataset('LineID2')
+order by $c.l_orderkey, $c.l_linenumber
+return $c
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_01/feeds_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_01/feeds_01.1.ddl.aql
index 4705cce..ef2df84 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_01/feeds_01.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_01/feeds_01.1.ddl.aql
@@ -15,8 +15,9 @@
timestamp : string
}
-create feed dataset TweetFeed(TweetType)
-using "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory"
-(("output-type-name"="TweetType"),("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("tuple-interval"="10"))
+create dataset Tweets(TweetType)
primary key id;
+create feed TweetFeed
+using file_feed
+(("output-type-name"="TweetType"),("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("tuple-interval"="10"))
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_01/feeds_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_01/feeds_01.3.query.aql
index 5ee2e87..392e471 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_01/feeds_01.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_01/feeds_01.3.query.aql
@@ -4,6 +4,6 @@
* Date : 24th Dec 2012
*/
-for $x in dataset('Metadata.Dataset')
-where $x.DataverseName='feeds' and $x.DatasetName='TweetFeed'
+for $x in dataset('Metadata.Feed')
+where $x.DataverseName='feeds' and $x.FeedName='TweetFeed'
return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.1.ddl.aql
index aafd2c9..fe5b88f 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.1.ddl.aql
@@ -9,17 +9,18 @@
create dataverse feeds;
use dataverse feeds;
+
create type TweetType as closed {
id: string,
username : string,
location : string,
text : string,
timestamp : string
-}
+};
-create feed dataset TweetFeed(TweetType)
-using "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory"
-(("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("output-type-name"="TweetType"),("tuple-interval"="10"))
+create dataset Tweets(TweetType)
primary key id;
-
+create feed TweetFeed
+using file_feed
+(("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("output-type-name"="TweetType"),("tuple-interval"="10"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.2.update.aql
index 01b0925..3f06b7a 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.2.update.aql
@@ -6,5 +6,7 @@
*/
use dataverse feeds;
-
-begin feed TweetFeed;
+
+set wait-for-completion-feed "true";
+
+connect feed TweetFeed to dataset Tweets;
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.3.query.aql
index 230aa40..3230002 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.3.query.aql
@@ -7,6 +7,6 @@
use dataverse feeds;
-for $x in dataset('TweetFeed')
+for $x in dataset('Tweets')
order by $x.id
return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.1.ddl.aql
index 488ed2f..8cdeb9b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.1.ddl.aql
@@ -16,13 +16,15 @@
timestamp : string
}
+create dataset Tweets(TweetType)
+primary key id;
+
create function feed_processor($x) {
$x
}
-create feed dataset TweetFeed(TweetType)
-using "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory"
+create feed TweetFeed
+using file_feed
(("output-type-name"="TweetType"),("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("tuple-interval"="10"))
-apply function feed_processor@1
-primary key id;
+apply function feed_processor@1;
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.3.query.aql
index 1922f39..a7ee344 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_03/feeds_03.3.query.aql
@@ -4,6 +4,6 @@
* Date : 24th Dec 2012
*/
-for $x in dataset('Metadata.Dataset')
-where $x.DataverseName='feeds' and $x.DatasetName='TweetFeed'
+for $x in dataset('Metadata.Feed')
+where $x.DataverseName='feeds' and $x.FeedName='TweetFeed'
return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.1.ddl.aql
index 326b2d5..6f539b0 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.1.ddl.aql
@@ -17,8 +17,9 @@
timestamp : string
}
-create feed dataset TweetFeed(TweetType)
-using "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory"
-(("fs"="hdfs"),("hdfs"="hdfs://127.0.0.1:31888"),("path"="/asterix/obamatweets.adm"),("format"="adm"),("input-format"="text-input-format"),("output-type-name"="TweetType"),("tuple-interval"="10"))
+create dataset Tweets(TweetType)
primary key id;
+create feed TweetFeed
+using file_feed
+(("fs"="hdfs"),("hdfs"="hdfs://127.0.0.1:31888"),("path"="/asterix/obamatweets.adm"),("format"="adm"),("input-format"="text-input-format"),("output-type-name"="TweetType"),("tuple-interval"="10"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.2.update.aql
index 060576e..e87111c 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.2.update.aql
@@ -8,4 +8,6 @@
use dataverse feeds;
-begin feed TweetFeed;
+set wait-for-completion-feed "true";
+
+connect feed TweetFeed to dataset Tweets;
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.3.query.aql
index 714dd80..451acde 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.3.query.aql
@@ -7,6 +7,6 @@
*/
use dataverse feeds;
-for $x in dataset('TweetFeed')
+for $x in dataset('Tweets')
order by $x.id
return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.1.ddl.aql
new file mode 100644
index 0000000..ec92972
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.1.ddl.aql
@@ -0,0 +1,39 @@
+/*
+ * Description : Create a feed dataset that uses the synthetic feed simulator adapter.
+ The synthetic feed simulator uses the Social-Data generator to generate data and simulate a feed. The feed lasts a configured duration with data arriving at a configured rate (tweets per second).
+ Verify the existence of data after the feed finishes.
+
+ * Expected Res : Success
+ * Date : 20th Jun 2013
+ */
+
+drop dataverse feeds if exists;
+create dataverse feeds;
+use dataverse feeds;
+
+create type TwitterUserType as closed {
+ screen-name: string,
+ lang: string,
+ friends_count: int32,
+ statuses_count: int32,
+ name: string,
+ followers_count: int32
+}
+
+create type TweetMessageType as closed {
+ tweetid: string,
+ user: TwitterUserType,
+ sender-location: point,
+ send-time: datetime,
+ referred-topics: {{ string }},
+ message-text: string
+}
+
+create dataset SyntheticTweets(TweetMessageType)
+primary key tweetid;
+
+
+create feed SyntheticTweetFeed
+using synthetic_twitter_feed
+(("duration"="5"),("tps"="2"),("dataverse-dataset"="feeds:SyntheticTweets"))
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.2.update.aql
new file mode 100644
index 0000000..d9e5404
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.2.update.aql
@@ -0,0 +1,15 @@
+/*
+ * Description : Create a feed dataset that uses the synthetic feed simulator adapter.
+ The synthetic feed simulator uses the Social-Data generator to generate data and simulate a feed.
+ The feed lasts a configured duration with data arriving at a configured rate (tweets per second).
+ Verify the existence of data after the feed finishes.
+
+ * Expected Res : Success
+ * Date : 20th Jun 2013
+ */
+
+use dataverse feeds;
+
+set wait-for-completion-feed "true";
+
+connect feed SyntheticTweetFeed to dataset SyntheticTweets;
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.3.query.aql
new file mode 100644
index 0000000..dd520da
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_05/feeds_05.3.query.aql
@@ -0,0 +1,21 @@
+/*
+ * Description : Create a feed dataset that uses the synthetic feed simulator adapter.
+ The synthetic feed simulator uses the Social-Data generator to generate data and simulate a feed.
+ The feed lasts a configured duration with data arriving at a configured rate (tweets per second).
+ Verify the existence of data after the feed finishes.
+
+ * Expected Res : Success
+ * Date : 20th Jun 2013
+ */
+
+use dataverse feeds;
+
+let $totalTweets:=count(
+for $x in dataset('SyntheticTweets')
+return $x)
+return
+(if($totalTweets > 0)
+ then 1
+else
+ 0
+)
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.1.ddl.aql
index 62ac61f..805a41c 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.1.ddl.aql
@@ -8,6 +8,7 @@
create dataverse feeds;
use dataverse feeds;
+
create type TweetType as closed {
id: string,
username : string,
@@ -16,8 +17,11 @@
timestamp : string
}
-create feed dataset TweetFeed(TweetType)
-using "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory"
-(("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("output-type-name"="TweetType"),("tuple-interval"="10"))
+create dataset Tweets(TweetType)
primary key id;
+create feed TweetFeed
+using file_feed
+(("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("output-type-name"="TweetType"),("tuple-interval"="10"));
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.2.update.aql
index 0e22d6e..fc71769 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.2.update.aql
@@ -6,5 +6,7 @@
*/
use dataverse feeds;
-
-begin feed feeds.TweetFeed;
+
+set wait-for-completion-feed "true";
+
+connect feed feeds.TweetFeed to dataset Tweets;
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.3.query.aql
index 48e18e2..227913d 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.3.query.aql
@@ -6,6 +6,6 @@
*/
use dataverse feeds;
-for $x in dataset('TweetFeed')
+for $x in dataset('Tweets')
order by $x.id
return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.1.ddl.aql
index ca09806..92f716d 100644
--- a/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.1.ddl.aql
@@ -17,9 +17,10 @@
timestamp : string
}
-create feed dataset TweetFeed(TweetType)
-using "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory"
-(("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("output-type-name"="TweetType"),("tuple-interval"="10"))
+create dataset Tweets(TweetType)
primary key id
hints(cardinality=200);
+create feed TweetFeed
+using file_feed
+(("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("output-type-name"="TweetType"),("tuple-interval"="10"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.2.update.aql
index 7851440..19e0930d 100644
--- a/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.2.update.aql
@@ -8,4 +8,6 @@
use dataverse feeds;
-begin feed feeds.TweetFeed;
+set wait-for-completion-feed "true";
+
+connect feed TweetFeed to dataset Tweets;
diff --git a/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.3.query.aql
index f1127f0..7ee0dcf 100644
--- a/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.3.query.aql
@@ -8,7 +8,7 @@
use dataverse feeds;
-for $x in dataset('TweetFeed')
+for $x in dataset('Tweets')
order by $x.id
return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.1.ddl.aql
deleted file mode 100644
index e69de29..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.1.ddl.aql
+++ /dev/null
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.2.update.aql
deleted file mode 100644
index e69de29..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.2.update.aql
+++ /dev/null
diff --git a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv02/cross-dv02.1.adm b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv02/cross-dv02.1.adm
index 89f66a8..2df9621 100644
--- a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv02/cross-dv02.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv02/cross-dv02.1.adm
@@ -1,4 +1,11 @@
+<<<<<<< HEAD
+{ "DataverseName": "student", "DatasetName": "gdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:04:06 PDT 2013", "DatasetId": 656, "PendingOp": 0 }
+{ "DataverseName": "student", "DatasetName": "ugdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:04:06 PDT 2013", "DatasetId": 655, "PendingOp": 0 }
+{ "DataverseName": "teacher", "DatasetName": "prof", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 657, "PendingOp": 0 }
+{ "DataverseName": "teacher", "DatasetName": "pstdoc", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 658, "PendingOp": 0 }
+=======
{ "DataverseName": "student", "DatasetName": "gdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:56 PDT 2013", "DatasetId": 662, "PendingOp": 0 }
{ "DataverseName": "student", "DatasetName": "ugdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:56 PDT 2013", "DatasetId": 661, "PendingOp": 0 }
{ "DataverseName": "teacher", "DatasetName": "prof", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:56 PDT 2013", "DatasetId": 663, "PendingOp": 0 }
-{ "DataverseName": "teacher", "DatasetName": "pstdoc", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:56 PDT 2013", "DatasetId": 664, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "teacher", "DatasetName": "pstdoc", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:56 PDT 2013", "DatasetId": 664, "PendingOp": 0 }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv04/cross-dv04.1.adm b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv04/cross-dv04.1.adm
index f9981d1..09e8d1e 100644
--- a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv04/cross-dv04.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv04/cross-dv04.1.adm
@@ -1,4 +1,11 @@
+<<<<<<< HEAD
+{ "DataverseName": "student", "DatasetName": "gdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 668, "PendingOp": 0 }
+{ "DataverseName": "student", "DatasetName": "ugdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 667, "PendingOp": 0 }
+{ "DataverseName": "teacher", "DatasetName": "prof", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 669, "PendingOp": 0 }
+{ "DataverseName": "teacher", "DatasetName": "pstdoc", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 670, "PendingOp": 0 }
+=======
{ "DataverseName": "student", "DatasetName": "gdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:57 PDT 2013", "DatasetId": 674, "PendingOp": 0 }
{ "DataverseName": "student", "DatasetName": "ugdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:57 PDT 2013", "DatasetId": 673, "PendingOp": 0 }
{ "DataverseName": "teacher", "DatasetName": "prof", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:57 PDT 2013", "DatasetId": 675, "PendingOp": 0 }
-{ "DataverseName": "teacher", "DatasetName": "pstdoc", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:57 PDT 2013", "DatasetId": 676, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "teacher", "DatasetName": "pstdoc", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:57 PDT 2013", "DatasetId": 676, "PendingOp": 0 }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
index 2d04f8b..d776957 100644
--- a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
@@ -1,7 +1,17 @@
+<<<<<<< HEAD
+{ "DataverseName": "test1", "DatasetName": "TwitterData", "DataTypeName": "Tweet", "DatasetType": "EXTERNAL", "InternalDetails": null, "ExternalDetails": { "DatasourceAdapter": "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter", "Properties": [ { "Name": "path", "Value": "nc1://data/twitter/extrasmalltweets.txt" }, { "Name": "format", "Value": "adm" } ] }, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 678, "PendingOp": 0 }
+{ "DataverseName": "test1", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 672, "PendingOp": 0 }
+{ "DataverseName": "test1", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 675, "PendingOp": 0 }
+{ "DataverseName": "test1", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 676, "PendingOp": 0 }
+{ "DataverseName": "test2", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 673, "PendingOp": 0 }
+{ "DataverseName": "test2", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 674, "PendingOp": 0 }
+{ "DataverseName": "test2", "DatasetName": "t4", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:04:07 PDT 2013", "DatasetId": 677, "PendingOp": 0 }
+=======
{ "DataverseName": "test1", "DatasetName": "TwitterData", "DataTypeName": "Tweet", "DatasetType": "EXTERNAL", "InternalDetails": null, "ExternalDetails": { "DatasourceAdapter": "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter", "Properties": [ { "Name": "path", "Value": "nc1://data/twitter/extrasmalltweets.txt" }, { "Name": "format", "Value": "adm" } ] }, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:58 PDT 2013", "DatasetId": 684, "PendingOp": 0 }
{ "DataverseName": "test1", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:58 PDT 2013", "DatasetId": 678, "PendingOp": 0 }
{ "DataverseName": "test1", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:58 PDT 2013", "DatasetId": 681, "PendingOp": 0 }
{ "DataverseName": "test1", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:58 PDT 2013", "DatasetId": 682, "PendingOp": 0 }
{ "DataverseName": "test2", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:58 PDT 2013", "DatasetId": 679, "PendingOp": 0 }
{ "DataverseName": "test2", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:58 PDT 2013", "DatasetId": 680, "PendingOp": 0 }
-{ "DataverseName": "test2", "DatasetName": "t4", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:58 PDT 2013", "DatasetId": 683, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "test2", "DatasetName": "t4", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:58 PDT 2013", "DatasetId": 683, "PendingOp": 0 }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/insert-record-function/insert-record-function.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/insert-record-function/insert-record-function.1.adm
new file mode 100644
index 0000000..3ea2b0c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/insert-record-function/insert-record-function.1.adm
@@ -0,0 +1,2 @@
+{ "l_orderkey": 1, "l_linenumber": 2, "l_suppkey": 3 }
+{ "l_orderkey": 2, "l_linenumber": 3, "l_suppkey": 4 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/feeds/feeds_01/feeds_01.1.adm b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_01/feeds_01.1.adm
index dec4c6c..824e8c7 100644
--- a/asterix-app/src/test/resources/runtimets/results/feeds/feeds_01/feeds_01.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_01/feeds_01.1.adm
@@ -1 +1,5 @@
-{ "DataverseName": "feeds", "DatasetName": "TweetFeed", "DataTypeName": "TweetType", "DatasetType": "FEED", "InternalDetails": null, "ExternalDetails": null, "FeedDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "DatasourceAdapter": "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory", "Properties": [ { "Name": "output-type-name", "Value": "TweetType" }, { "Name": "fs", "Value": "localfs" }, { "Name": "path", "Value": "nc1://data/twitter/obamatweets.adm" }, { "Name": "format", "Value": "adm" }, { "Name": "tuple-interval", "Value": "10" } ], "Function": null, "Status": "INACTIVE", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:44:30 PDT 2013", "DatasetId": 705, "PendingOp": 0 }
\ No newline at end of file
+<<<<<<< HEAD
+{ "DataverseName": "feeds", "FeedName": "TweetFeed", "AdaptorName": "file_feed", "AdaptorConfiguration": {{ { "Name": "output-type-name", "Value": "TweetType" }, { "Name": "fs", "Value": "localfs" }, { "Name": "path", "Value": "nc1://data/twitter/obamatweets.adm" }, { "Name": "format", "Value": "adm" }, { "Name": "tuple-interval", "Value": "10" } }}, "Function": null, "Timestamp": "Wed Jul 17 08:26:24 PDT 2013" }
+=======
+{ "DataverseName": "feeds", "DatasetName": "TweetFeed", "DataTypeName": "TweetType", "DatasetType": "FEED", "InternalDetails": null, "ExternalDetails": null, "FeedDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "DatasourceAdapter": "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory", "Properties": [ { "Name": "output-type-name", "Value": "TweetType" }, { "Name": "fs", "Value": "localfs" }, { "Name": "path", "Value": "nc1://data/twitter/obamatweets.adm" }, { "Name": "format", "Value": "adm" }, { "Name": "tuple-interval", "Value": "10" } ], "Function": null, "Status": "INACTIVE", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:44:30 PDT 2013", "DatasetId": 705, "PendingOp": 0 }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/runtimets/results/feeds/feeds_03/feeds_03.1.adm b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_03/feeds_03.1.adm
index 3fa1d10..2844787 100644
--- a/asterix-app/src/test/resources/runtimets/results/feeds/feeds_03/feeds_03.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_03/feeds_03.1.adm
@@ -1 +1,5 @@
-{ "DataverseName": "feeds", "DatasetName": "TweetFeed", "DataTypeName": "TweetType", "DatasetType": "FEED", "InternalDetails": null, "ExternalDetails": null, "FeedDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "DatasourceAdapter": "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory", "Properties": [ { "Name": "output-type-name", "Value": "TweetType" }, { "Name": "fs", "Value": "localfs" }, { "Name": "path", "Value": "nc1://data/twitter/obamatweets.adm" }, { "Name": "format", "Value": "adm" }, { "Name": "tuple-interval", "Value": "10" } ], "Function": "feeds.feed_processor@1", "Status": "INACTIVE", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:44:30 PDT 2013", "DatasetId": 707, "PendingOp": 0 }
\ No newline at end of file
+<<<<<<< HEAD
+{ "DataverseName": "feeds", "FeedName": "TweetFeed", "AdaptorName": "file_feed", "AdaptorConfiguration": {{ { "Name": "output-type-name", "Value": "TweetType" }, { "Name": "fs", "Value": "localfs" }, { "Name": "path", "Value": "nc1://data/twitter/obamatweets.adm" }, { "Name": "format", "Value": "adm" }, { "Name": "tuple-interval", "Value": "10" } }}, "Function": "feeds.feed_processor@1", "Timestamp": "Wed Jul 17 08:06:52 PDT 2013" }
+=======
+{ "DataverseName": "feeds", "DatasetName": "TweetFeed", "DataTypeName": "TweetType", "DatasetType": "FEED", "InternalDetails": null, "ExternalDetails": null, "FeedDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "DatasourceAdapter": "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory", "Properties": [ { "Name": "output-type-name", "Value": "TweetType" }, { "Name": "fs", "Value": "localfs" }, { "Name": "path", "Value": "nc1://data/twitter/obamatweets.adm" }, { "Name": "format", "Value": "adm" }, { "Name": "tuple-interval", "Value": "10" } ], "Function": "feeds.feed_processor@1", "Status": "INACTIVE", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:44:30 PDT 2013", "DatasetId": 707, "PendingOp": 0 }
+>>>>>>> master
diff --git a/asterix-app/src/test/resources/runtimets/results/feeds/feeds_05/feeds_05.1.adm b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_05/feeds_05.1.adm
new file mode 100644
index 0000000..d00491f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_05/feeds_05.1.adm
@@ -0,0 +1 @@
+1
diff --git a/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm
index c084678..475962d 100644
--- a/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm
@@ -1,3 +1,9 @@
+{ "DataverseName": "DMLTest", "DatasetName": "FacebookUsers1", "DataTypeName": "FacebookUserType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:03:45 PDT 2013", "DatasetId": 369, "PendingOp": 0 }
+{ "DataverseName": "DMLTest", "DatasetName": "FacebookUsers2", "DataTypeName": "FacebookUserType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:03:45 PDT 2013", "DatasetId": 370, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:03:01 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:03:01 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:03:01 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Tue Jul 16 23:03:01 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
{ "DataverseName": "DMLTest", "DatasetName": "FacebookUsers1", "DataTypeName": "FacebookUserType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:07 PDT 2013", "DatasetId": 375, "PendingOp": 0 }
{ "DataverseName": "DMLTest", "DatasetName": "FacebookUsers2", "DataTypeName": "FacebookUserType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:26:07 PDT 2013", "DatasetId": 376, "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "CompactionPolicy" ], "PrimaryKey": [ "DataverseName", "CompactionPolicy" ], "GroupName": "MetadataGroup", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolernace-component-count", "Value": "5" } ] }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{ }}, "Timestamp": "Sun Sep 22 22:24:38 PDT 2013", "DatasetId": 9, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index 3abacd5..7c614c0 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -4350,6 +4350,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="feeds">
+ <compilation-unit name="feeds_05">
+ <output-dir compare="Text">feeds_05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
<compilation-unit name="issue_230_feeds">
<output-dir compare="Text">issue_230_feeds</output-dir>
</compilation-unit>
diff --git a/asterix-aql/pom.xml b/asterix-aql/pom.xml
index 465c3b9..4b83e1c 100644
--- a/asterix-aql/pom.xml
+++ b/asterix-aql/pom.xml
@@ -115,6 +115,12 @@
</build>
<dependencies>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <version>4.8.1</version>
+ <scope>test</scope>
+ </dependency>
<dependency>
<groupId>edu.uci.ics.asterix</groupId>
<artifactId>asterix-common</artifactId>
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/base/Statement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/base/Statement.java
index 34693e4..ca66a97 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/base/Statement.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/base/Statement.java
@@ -37,8 +37,10 @@
INDEX_DECL,
CREATE_DATAVERSE,
INDEX_DROP,
- BEGIN_FEED,
- CONTROL_FEED,
+ CREATE_FEED,
+ DROP_FEED,
+ CONNECT_FEED,
+ DISCONNECT_FEED,
CREATE_FUNCTION,
FUNCTION_DROP,
COMPACT
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/BeginFeedStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/BeginFeedStatement.java
deleted file mode 100644
index a119e2a..0000000
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/BeginFeedStatement.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.aql.expression;
-
-import java.io.StringReader;
-import java.util.List;
-
-import edu.uci.ics.asterix.aql.base.Statement;
-import edu.uci.ics.asterix.aql.expression.visitor.IAqlExpressionVisitor;
-import edu.uci.ics.asterix.aql.expression.visitor.IAqlVisitorWithVoidReturn;
-import edu.uci.ics.asterix.aql.parser.AQLParser;
-import edu.uci.ics.asterix.aql.parser.ParseException;
-import edu.uci.ics.asterix.aql.util.FunctionUtils;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.functions.FunctionConstants;
-import edu.uci.ics.asterix.common.functions.FunctionSignature;
-import edu.uci.ics.asterix.metadata.MetadataException;
-import edu.uci.ics.asterix.metadata.MetadataManager;
-import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
-import edu.uci.ics.asterix.metadata.entities.Dataset;
-import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
-import edu.uci.ics.asterix.metadata.entities.Function;
-
-public class BeginFeedStatement implements Statement {
-
- private final Identifier dataverseName;
- private final Identifier datasetName;
- private Query query;
- private int varCounter;
-
- public BeginFeedStatement(Identifier dataverseName, Identifier datasetName, int varCounter) {
- this.dataverseName = dataverseName;
- this.datasetName = datasetName;
- this.varCounter = varCounter;
- }
-
- public void initialize(MetadataTransactionContext mdTxnCtx, Dataset dataset) throws MetadataException {
- query = new Query();
- FeedDatasetDetails feedDetails = (FeedDatasetDetails) dataset.getDatasetDetails();
- String functionName = feedDetails.getFunction() == null ? null : feedDetails.getFunction().getName();
- StringBuilder builder = new StringBuilder();
- builder.append("set" + " " + FunctionUtils.IMPORT_PRIVATE_FUNCTIONS + " " + "'" + Boolean.TRUE + "'" + ";\n");
- builder.append("insert into dataset " + datasetName + " ");
-
- if (functionName == null) {
- builder.append(" (" + " for $x in feed-ingest ('" + datasetName + "') ");
- builder.append(" return $x");
- } else {
- int arity = feedDetails.getFunction().getArity();
- FunctionSignature signature = new FunctionSignature(dataset.getDataverseName(), functionName, arity);
- Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, signature);
- if (function == null) {
- throw new MetadataException(" Unknown function " + feedDetails.getFunction());
- }
- if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)) {
- String param = function.getParams().get(0);
- builder.append(" (" + " for" + " " + param + " in feed-ingest ('" + datasetName + "') ");
- builder.append(" let $y:=(" + function.getFunctionBody() + ")" + " return $y");
- } else {
- builder.append(" (" + " for $x in feed-ingest ('" + datasetName + "') ");
- builder.append(" let $y:=" + function.getName() + "(" + "$x" + ")");
- builder.append(" return $y");
- }
-
- }
- builder.append(")");
- builder.append(";");
- AQLParser parser = new AQLParser(new StringReader(builder.toString()));
-
- List<Statement> statements;
- try {
- statements = parser.Statement();
- query = ((InsertStatement) statements.get(1)).getQuery();
- } catch (ParseException pe) {
- throw new MetadataException(pe);
- }
-
- }
-
- public Identifier getDataverseName() {
- return dataverseName;
- }
-
- public Identifier getDatasetName() {
- return datasetName;
- }
-
- public Query getQuery() {
- return query;
- }
-
- public int getVarCounter() {
- return varCounter;
- }
-
- @Override
- public Kind getKind() {
- return Kind.BEGIN_FEED;
- }
-
- @Override
- public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException {
- return visitor.visitBeginFeedStatement(this, arg);
- }
-
- @Override
- public <T> void accept(IAqlVisitorWithVoidReturn<T> visitor, T arg) throws AsterixException {
- visitor.visit(this, arg);
- }
-
-}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ConnectFeedStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ConnectFeedStatement.java
new file mode 100644
index 0000000..ec8b6be
--- /dev/null
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ConnectFeedStatement.java
@@ -0,0 +1,184 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.aql.expression;
+
+import java.io.StringReader;
+import java.util.List;
+
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlExpressionVisitor;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlVisitorWithVoidReturn;
+import edu.uci.ics.asterix.aql.parser.AQLParser;
+import edu.uci.ics.asterix.aql.parser.ParseException;
+import edu.uci.ics.asterix.aql.util.FunctionUtils;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Feed;
+import edu.uci.ics.asterix.metadata.entities.Function;
+import edu.uci.ics.asterix.metadata.feeds.BuiltinFeedPolicies;
+import edu.uci.ics.asterix.metadata.feeds.FeedUtil;
+import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+
+public class ConnectFeedStatement implements Statement {
+
+ private final Identifier dataverseName;
+ private final Identifier datasetName;
+ private final String feedName;
+ private final String policy;
+ private Query query;
+ private int varCounter;
+ private boolean forceConnect = false;
+
+ public static final String WAIT_FOR_COMPLETION = "wait-for-completion-feed";
+
+ public ConnectFeedStatement(Pair<Identifier, Identifier> feedNameCmp, Pair<Identifier, Identifier> datasetNameCmp,
+ String policy, int varCounter) {
+ if (feedNameCmp.first != null && datasetNameCmp.first != null
+ && !feedNameCmp.first.getValue().equals(datasetNameCmp.first.getValue())) {
+ throw new IllegalArgumentException("Dataverse for source feed and target dataset do not match");
+ }
+ this.dataverseName = feedNameCmp.first != null ? feedNameCmp.first
+ : datasetNameCmp.first != null ? datasetNameCmp.first : null;
+ this.datasetName = datasetNameCmp.second;
+ this.feedName = feedNameCmp.second.getValue();
+ this.policy = policy != null ? policy : BuiltinFeedPolicies.DEFAULT_POLICY.getPolicyName();
+ this.varCounter = varCounter;
+ }
+
+ public ConnectFeedStatement(Identifier dataverseName, Identifier feedName, Identifier datasetName, String policy,
+ int varCounter) {
+ this.dataverseName = dataverseName;
+ this.datasetName = datasetName;
+ this.feedName = feedName.getValue();
+ this.policy = policy != null ? policy : BuiltinFeedPolicies.DEFAULT_POLICY.getPolicyName();
+ this.varCounter = varCounter;
+ }
+
+ public void initialize(MetadataTransactionContext mdTxnCtx, Dataset targetDataset, Feed sourceFeed)
+ throws MetadataException {
+ query = new Query();
+ FunctionSignature appliedFunction = sourceFeed.getAppliedFunction();
+ Function function = null;
+ String adapterOutputType = null;
+ if (appliedFunction != null) {
+ function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, appliedFunction);
+ if (function == null) {
+ throw new MetadataException(" Unknown function " + function);
+ } else if (function.getParams().size() > 1) {
+ throw new MetadataException(" Incompatible function: " + appliedFunction
+ + " Number if arguments must be 1");
+ }
+ }
+
+ org.apache.commons.lang3.tuple.Pair<IAdapterFactory, ARecordType> factoryOutput = null;
+ try {
+ factoryOutput = FeedUtil.getFeedFactoryAndOutput(sourceFeed, mdTxnCtx);
+ adapterOutputType = factoryOutput.getRight().getTypeName();
+ } catch (AlgebricksException ae) {
+ throw new MetadataException(ae);
+ }
+
+ StringBuilder builder = new StringBuilder();
+ builder.append("set" + " " + FunctionUtils.IMPORT_PRIVATE_FUNCTIONS + " " + "'" + Boolean.TRUE + "'" + ";\n");
+ builder.append("insert into dataset " + datasetName + " ");
+
+ if (appliedFunction == null) {
+ builder.append(" (" + " for $x in feed-ingest ('" + feedName + "'" + "," + "'" + adapterOutputType + "'"
+ + "," + "'" + targetDataset.getDatasetName() + "'" + ")");
+ builder.append(" return $x");
+ } else {
+ if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)) {
+ String param = function.getParams().get(0);
+ builder.append(" (" + " for" + " " + param + " in feed-ingest ('" + feedName + "'" + "," + "'"
+ + adapterOutputType + "'" + "," + "'" + targetDataset.getDatasetName() + "'" + ")");
+ builder.append(" let $y:=(" + function.getFunctionBody() + ")" + " return $y");
+ } else {
+ builder.append(" (" + " for $x in feed-ingest ('" + feedName + "'" + "," + "'" + adapterOutputType
+ + "'" + "," + "'" + targetDataset.getDatasetName() + "'" + ")");
+ builder.append(" let $y:=" + sourceFeed.getDataverseName() + "." + function.getName() + "(" + "$x"
+ + ")");
+ builder.append(" return $y");
+ }
+
+ }
+ builder.append(")");
+ builder.append(";");
+ AQLParser parser = new AQLParser(new StringReader(builder.toString()));
+
+ List<Statement> statements;
+ try {
+ statements = parser.Statement();
+ query = ((InsertStatement) statements.get(1)).getQuery();
+ } catch (ParseException pe) {
+ throw new MetadataException(pe);
+ }
+
+ }
+
+ public Identifier getDataverseName() {
+ return dataverseName;
+ }
+
+ public Identifier getDatasetName() {
+ return datasetName;
+ }
+
+ public Query getQuery() {
+ return query;
+ }
+
+ public int getVarCounter() {
+ return varCounter;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.CONNECT_FEED;
+ }
+
+ public String getPolicy() {
+ return policy;
+ }
+
+ @Override
+ public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException {
+ return visitor.visitConnectFeedStatement(this, arg);
+ }
+
+ @Override
+ public <T> void accept(IAqlVisitorWithVoidReturn<T> visitor, T arg) throws AsterixException {
+ visitor.visit(this, arg);
+ }
+
+ public boolean forceConnect() {
+ return forceConnect;
+ }
+
+ public void setForceConnect(boolean forceConnect) {
+ this.forceConnect = forceConnect;
+ }
+
+ public String getFeedName() {
+ return feedName;
+ }
+
+}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ControlFeedStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ControlFeedStatement.java
deleted file mode 100644
index d442fe7..0000000
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ControlFeedStatement.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.aql.expression;
-
-import java.util.Map;
-
-import edu.uci.ics.asterix.aql.base.Statement;
-import edu.uci.ics.asterix.aql.expression.visitor.IAqlExpressionVisitor;
-import edu.uci.ics.asterix.aql.expression.visitor.IAqlVisitorWithVoidReturn;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-
-public class ControlFeedStatement implements Statement {
-
- private final Identifier dataverseName;
- private final Identifier datasetName;
-
- public enum OperationType {
- BEGIN,
- SUSPEND,
- RESUME,
- END,
- ALTER
- }
-
- private OperationType operationType;
- private Map<String, String> alterAdapterConfParams;
-
- public ControlFeedStatement(OperationType operation, Identifier dataverseName, Identifier datasetName) {
- this.operationType = operation;
- this.datasetName = datasetName;
- this.dataverseName = dataverseName;
- }
-
- public ControlFeedStatement(OperationType operation, Identifier dataverseName, Identifier datasetName,
- Map<String, String> alterAdapterConfParams) {
- this.operationType = operation;
- this.datasetName = datasetName;
- this.dataverseName = dataverseName;
- this.alterAdapterConfParams = alterAdapterConfParams;
- }
-
- public Identifier getDataverseName() {
- return dataverseName;
- }
-
- public Identifier getDatasetName() {
- return datasetName;
- }
-
- public OperationType getOperationType() {
- return operationType;
- }
-
- public void setOperation(OperationType operationType) {
- this.operationType = operationType;
- }
-
- @Override
- public Kind getKind() {
- return Kind.CONTROL_FEED;
- }
-
- public Map<String, String> getAlterAdapterConfParams() {
- return alterAdapterConfParams;
- }
-
- @Override
- public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException {
- return visitor.visitControlFeedStatement(this, arg);
- }
-
- @Override
- public <T> void accept(IAqlVisitorWithVoidReturn<T> visitor, T arg) throws AsterixException {
- visitor.visit(this, arg);
- }
-
-}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreateFeedStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreateFeedStatement.java
new file mode 100644
index 0000000..8f90e87
--- /dev/null
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreateFeedStatement.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.aql.expression;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlExpressionVisitor;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlVisitorWithVoidReturn;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
+
+public class CreateFeedStatement implements Statement {
+
+ private final Identifier dataverseName;
+ private final Identifier feedName;
+ private final String adaptorName;
+ private final Map<String, String> adaptorConfiguration;
+ private final FunctionSignature appliedFunction;
+ private final boolean ifNotExists;
+
+ public CreateFeedStatement(Identifier dataverseName, Identifier feedName, String adaptorName,
+ Map<String, String> adaptorConfiguration, FunctionSignature appliedFunction, boolean ifNotExists) {
+ this.feedName = feedName;
+ this.dataverseName = dataverseName;
+ this.adaptorName = adaptorName;
+ this.adaptorConfiguration = adaptorConfiguration;
+ this.appliedFunction = appliedFunction;
+ this.ifNotExists = ifNotExists;
+ }
+
+ public Identifier getDataverseName() {
+ return dataverseName;
+ }
+
+ public Identifier getFeedName() {
+ return feedName;
+ }
+
+ public String getAdaptorName() {
+ return adaptorName;
+ }
+
+ public Map<String, String> getAdaptorConfiguration() {
+ return adaptorConfiguration;
+ }
+
+ public FunctionSignature getAppliedFunction() {
+ return appliedFunction;
+ }
+
+ public boolean getIfNotExists() {
+ return this.ifNotExists;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.CREATE_FEED;
+ }
+
+ @Override
+ public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException {
+ return visitor.visitCreateFeedStatement(this, arg);
+ }
+
+ @Override
+ public <T> void accept(IAqlVisitorWithVoidReturn<T> visitor, T arg) throws AsterixException {
+ visitor.visit(this, arg);
+ }
+
+}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/DisconnectFeedStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/DisconnectFeedStatement.java
new file mode 100644
index 0000000..b2035a6
--- /dev/null
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/DisconnectFeedStatement.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.aql.expression;
+
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlExpressionVisitor;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlVisitorWithVoidReturn;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+
+public class DisconnectFeedStatement implements Statement {
+
+ private final Identifier dataverseName;
+ private final Identifier feedName;
+ private final Identifier datasetName;
+
+ public DisconnectFeedStatement(Identifier dataverseName, Identifier feedName, Identifier datasetName) {
+ this.feedName = feedName;
+ this.datasetName = datasetName;
+ this.dataverseName = dataverseName;
+ }
+
+ public DisconnectFeedStatement(Pair<Identifier, Identifier> feedNameComponent,
+ Pair<Identifier, Identifier> datasetNameComponent) {
+ if (feedNameComponent.first != null && datasetNameComponent.first != null
+ && !feedNameComponent.first.getValue().equals(datasetNameComponent.first.getValue())) {
+ throw new IllegalArgumentException("Dataverse for source feed and target dataset do not match");
+ }
+ this.dataverseName = feedNameComponent.first != null ? feedNameComponent.first
+ : datasetNameComponent.first != null ? datasetNameComponent.first : null;
+ this.datasetName = datasetNameComponent.second;
+ this.feedName = feedNameComponent.second;
+ }
+
+ public Identifier getDataverseName() {
+ return dataverseName;
+ }
+
+ public Identifier getFeedName() {
+ return feedName;
+ }
+
+ public Identifier getDatasetName() {
+ return datasetName;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.DISCONNECT_FEED;
+ }
+
+ @Override
+ public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException {
+ return visitor.visitDisconnectFeedStatement(this, arg);
+ }
+
+ @Override
+ public <T> void accept(IAqlVisitorWithVoidReturn<T> visitor, T arg) throws AsterixException {
+ visitor.visit(this, arg);
+ }
+
+ @Override
+ public String toString() {
+ return "disconnect feed " + feedName + " from " + datasetName;
+ }
+
+}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/FeedDropStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/FeedDropStatement.java
new file mode 100644
index 0000000..0120a7d
--- /dev/null
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/FeedDropStatement.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.aql.expression;
+
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlExpressionVisitor;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlVisitorWithVoidReturn;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+
+public class FeedDropStatement implements Statement {
+
+ private final Identifier dataverseName;
+ private final Identifier feedName;
+ private boolean ifExists;
+
+ public FeedDropStatement(Identifier dataverseName, Identifier feedName, boolean ifExists) {
+ this.dataverseName = dataverseName;
+ this.feedName = feedName;
+ this.ifExists = ifExists;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.DROP_FEED;
+ }
+
+ public Identifier getDataverseName() {
+ return dataverseName;
+ }
+
+ public Identifier getFeedName() {
+ return feedName;
+ }
+
+ public boolean getIfExists() {
+ return ifExists;
+ }
+
+ @Override
+ public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException {
+ return visitor.visitDropFeedStatement(this, arg);
+ }
+
+ @Override
+ public <T> void accept(IAqlVisitorWithVoidReturn<T> visitor, T arg) throws AsterixException {
+ visitor.visit(this, arg);
+ }
+
+}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/AQLPrintVisitor.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/AQLPrintVisitor.java
index aac2b6b..22b63fa 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/AQLPrintVisitor.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/AQLPrintVisitor.java
@@ -21,11 +21,12 @@
import edu.uci.ics.asterix.aql.base.Clause;
import edu.uci.ics.asterix.aql.base.Expression;
import edu.uci.ics.asterix.aql.base.Literal;
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
import edu.uci.ics.asterix.aql.expression.CallExpr;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.CompactStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
import edu.uci.ics.asterix.aql.expression.DatasetDecl;
@@ -35,6 +36,7 @@
import edu.uci.ics.asterix.aql.expression.DistinctClause;
import edu.uci.ics.asterix.aql.expression.DropStatement;
import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
+import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
import edu.uci.ics.asterix.aql.expression.FieldAccessor;
import edu.uci.ics.asterix.aql.expression.FieldBinding;
import edu.uci.ics.asterix.aql.expression.ForClause;
@@ -82,462 +84,517 @@
import edu.uci.ics.asterix.common.exceptions.AsterixException;
public class AQLPrintVisitor implements IAqlVisitorWithVoidReturn<Integer> {
- // private int level =0;
- private final PrintWriter out;
+ // private int level =0;
+ private final PrintWriter out;
- public AQLPrintVisitor() {
- out = new PrintWriter(System.out);
- }
+ public AQLPrintVisitor() {
+ out = new PrintWriter(System.out);
+ }
- public AQLPrintVisitor(PrintWriter out) {
- this.out = out;
- }
+ public AQLPrintVisitor(PrintWriter out) {
+ this.out = out;
+ }
- private String skip(int step) {
- StringBuffer sb = new StringBuffer();
- for (int i = 0; i < step; i++)
- sb.append(" ");
- return sb.toString();
- }
+ private String skip(int step) {
+ StringBuffer sb = new StringBuffer();
+ for (int i = 0; i < step; i++)
+ sb.append(" ");
+ return sb.toString();
+ }
- @Override
- public void visit(Query q, Integer step) throws AsterixException {
- if (q.getBody() != null) {
- out.println("Query:");
- q.getBody().accept(this, step);
- } else {
- out.println("No query.");
- }
- }
+ @Override
+ public void visit(Query q, Integer step) throws AsterixException {
+ if (q.getBody() != null) {
+ out.println("Query:");
+ q.getBody().accept(this, step);
+ } else {
+ out.println("No query.");
+ }
+ }
- @Override
- public void visit(LiteralExpr l, Integer step) {
- Literal lc = l.getValue();
- if (lc.getLiteralType().equals(Literal.Type.TRUE) || lc.getLiteralType().equals(Literal.Type.FALSE)
- || lc.getLiteralType().equals(Literal.Type.NULL)) {
- out.println(skip(step) + "LiteralExpr [" + l.getValue().getLiteralType() + "]");
- } else {
- out.println(skip(step) + "LiteralExpr [" + l.getValue().getLiteralType() + "] ["
- + l.getValue().getStringValue() + "] ");
- }
- }
+ @Override
+ public void visit(LiteralExpr l, Integer step) {
+ Literal lc = l.getValue();
+ if (lc.getLiteralType().equals(Literal.Type.TRUE)
+ || lc.getLiteralType().equals(Literal.Type.FALSE)
+ || lc.getLiteralType().equals(Literal.Type.NULL)) {
+ out.println(skip(step) + "LiteralExpr ["
+ + l.getValue().getLiteralType() + "]");
+ } else {
+ out.println(skip(step) + "LiteralExpr ["
+ + l.getValue().getLiteralType() + "] ["
+ + l.getValue().getStringValue() + "] ");
+ }
+ }
- @Override
- public void visit(VariableExpr v, Integer step) {
- out.println(skip(step) + "Variable [ Name=" + v.getVar().getValue() + " Id=" + v.getVar().getId() + " ]");
- }
+ @Override
+ public void visit(VariableExpr v, Integer step) {
+ out.println(skip(step) + "Variable [ Name=" + v.getVar().getValue()
+ + " Id=" + v.getVar().getId() + " ]");
+ }
- @Override
- public void visit(ListConstructor lc, Integer step) throws AsterixException {
- boolean ordered = false;
- if (lc.getType().equals(ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR)) {
- ordered = true;
- }
+ @Override
+ public void visit(ListConstructor lc, Integer step) throws AsterixException {
+ boolean ordered = false;
+ if (lc.getType().equals(ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR)) {
+ ordered = true;
+ }
- out.println(skip(step) + (ordered == true ? "OrderedListConstructor " : "UnorderedListConstructor ") + "[");
- for (Expression e : lc.getExprList()) {
- e.accept(this, step + 1);
- }
- out.println(skip(step) + "]");
- }
+ out.println(skip(step)
+ + (ordered == true ? "OrderedListConstructor "
+ : "UnorderedListConstructor ") + "[");
+ for (Expression e : lc.getExprList()) {
+ e.accept(this, step + 1);
+ }
+ out.println(skip(step) + "]");
+ }
- @Override
- public void visit(RecordConstructor rc, Integer step) throws AsterixException {
- out.println(skip(step) + "RecordConstructor [");
- // fbList accept visitor
- for (FieldBinding fb : rc.getFbList()) {
- out.println(skip(step + 1) + "(");
- fb.getLeftExpr().accept(this, step + 2);
- out.println(skip(step + 2) + ":");
- fb.getRightExpr().accept(this, step + 2);
- out.println(skip(step + 1) + ")");
- }
- out.println(skip(step) + "]");
- }
+ @Override
+ public void visit(RecordConstructor rc, Integer step)
+ throws AsterixException {
+ out.println(skip(step) + "RecordConstructor [");
+ // fbList accept visitor
+ for (FieldBinding fb : rc.getFbList()) {
+ out.println(skip(step + 1) + "(");
+ fb.getLeftExpr().accept(this, step + 2);
+ out.println(skip(step + 2) + ":");
+ fb.getRightExpr().accept(this, step + 2);
+ out.println(skip(step + 1) + ")");
+ }
+ out.println(skip(step) + "]");
+ }
- @Override
- public void visit(CallExpr pf, Integer step) throws AsterixException {
- out.println(skip(step) + "FunctionCall " + pf.getFunctionSignature().toString() + "[");
- for (Expression expr : pf.getExprList()) {
- expr.accept(this, step + 1);
- }
- out.println(skip(step) + "]");
- }
+ @Override
+ public void visit(CallExpr pf, Integer step) throws AsterixException {
+ out.println(skip(step) + "FunctionCall "
+ + pf.getFunctionSignature().toString() + "[");
+ for (Expression expr : pf.getExprList()) {
+ expr.accept(this, step + 1);
+ }
+ out.println(skip(step) + "]");
+ }
- @Override
- public void visit(OperatorExpr ifbo, Integer step) throws AsterixException {
- List<Expression> exprList = ifbo.getExprList();
- List<OperatorType> opList = ifbo.getOpList();
- if (ifbo.isCurrentop()) {
- out.println(skip(step) + "OperatorExpr [");
- exprList.get(0).accept(this, step + 1);
- for (int i = 1; i < exprList.size(); i++) {
- out.println(skip(step + 1) + opList.get(i - 1));
- exprList.get(i).accept(this, step + 1);
- }
- out.println(skip(step) + "]");
- } else {
- exprList.get(0).accept(this, step);
- }
+ @Override
+ public void visit(OperatorExpr ifbo, Integer step) throws AsterixException {
+ List<Expression> exprList = ifbo.getExprList();
+ List<OperatorType> opList = ifbo.getOpList();
+ if (ifbo.isCurrentop()) {
+ out.println(skip(step) + "OperatorExpr [");
+ exprList.get(0).accept(this, step + 1);
+ for (int i = 1; i < exprList.size(); i++) {
+ out.println(skip(step + 1) + opList.get(i - 1));
+ exprList.get(i).accept(this, step + 1);
+ }
+ out.println(skip(step) + "]");
+ } else {
+ exprList.get(0).accept(this, step);
+ }
- }
+ }
- @Override
- public void visit(IfExpr ifexpr, Integer step) throws AsterixException {
- out.println(skip(step) + "IfExpr [");
- out.println(skip(step + 1) + "Condition:");
- ifexpr.getCondExpr().accept(this, step + 2);
- out.println(skip(step + 1) + "Then:");
- ifexpr.getThenExpr().accept(this, step + 2);
- out.println(skip(step + 1) + "Else:");
- ifexpr.getElseExpr().accept(this, step + 2);
- out.println(skip(step) + "]");
- }
+ @Override
+ public void visit(IfExpr ifexpr, Integer step) throws AsterixException {
+ out.println(skip(step) + "IfExpr [");
+ out.println(skip(step + 1) + "Condition:");
+ ifexpr.getCondExpr().accept(this, step + 2);
+ out.println(skip(step + 1) + "Then:");
+ ifexpr.getThenExpr().accept(this, step + 2);
+ out.println(skip(step + 1) + "Else:");
+ ifexpr.getElseExpr().accept(this, step + 2);
+ out.println(skip(step) + "]");
+ }
- @Override
- public void visit(FLWOGRExpression flwor, Integer step) throws AsterixException {
- out.println(skip(step) + "FLWOGR [");
- for (Clause cl : flwor.getClauseList()) {
- cl.accept(this, step + 1);
- }
- out.println(skip(step + 1) + "Return");
- flwor.getReturnExpr().accept(this, step + 2);
- out.println(skip(step) + "]");
- }
+ @Override
+ public void visit(FLWOGRExpression flwor, Integer step)
+ throws AsterixException {
+ out.println(skip(step) + "FLWOGR [");
+ for (Clause cl : flwor.getClauseList()) {
+ cl.accept(this, step + 1);
+ }
+ out.println(skip(step + 1) + "Return");
+ flwor.getReturnExpr().accept(this, step + 2);
+ out.println(skip(step) + "]");
+ }
- @Override
- public void visit(QuantifiedExpression qe, Integer step) throws AsterixException {
- out.println(skip(step) + "QuantifiedExpression " + qe.getQuantifier() + " [");
- // quantifiedList accept visitor
- for (QuantifiedPair pair : qe.getQuantifiedList()) {
- out.print(skip(step + 1) + "[");
- pair.getVarExpr().accept(this, 0);
- out.println(skip(step + 1) + "In");
- pair.getExpr().accept(this, step + 2);
- out.println(skip(step + 1) + "]");
- }
- out.println(skip(step + 1) + "Satifies [");
- qe.getSatisfiesExpr().accept(this, step + 2);
- out.println(skip(step + 1) + "]");// for satifies
- out.println(skip(step) + "]");// for quantifiedExpr
- }
+ @Override
+ public void visit(QuantifiedExpression qe, Integer step)
+ throws AsterixException {
+ out.println(skip(step) + "QuantifiedExpression " + qe.getQuantifier()
+ + " [");
+ // quantifiedList accept visitor
+ for (QuantifiedPair pair : qe.getQuantifiedList()) {
+ out.print(skip(step + 1) + "[");
+ pair.getVarExpr().accept(this, 0);
+ out.println(skip(step + 1) + "In");
+ pair.getExpr().accept(this, step + 2);
+ out.println(skip(step + 1) + "]");
+ }
+ out.println(skip(step + 1) + "Satifies [");
+ qe.getSatisfiesExpr().accept(this, step + 2);
+ out.println(skip(step + 1) + "]");// for satifies
+ out.println(skip(step) + "]");// for quantifiedExpr
+ }
- @Override
- public void visit(ForClause fc, Integer step) throws AsterixException {
- out.print(skip(step) + "For ");
- fc.getVarExpr().accept(this, 0);
- out.println(skip(step + 1) + "In ");
- fc.getInExpr().accept(this, step + 1);
- }
+ @Override
+ public void visit(ForClause fc, Integer step) throws AsterixException {
+ out.print(skip(step) + "For ");
+ fc.getVarExpr().accept(this, 0);
+ out.println(skip(step + 1) + "In ");
+ fc.getInExpr().accept(this, step + 1);
+ }
- @Override
- public void visit(LetClause lc, Integer step) throws AsterixException {
- out.print(skip(step) + "Let ");
- lc.getVarExpr().accept(this, 0);
- out.println(skip(step + 1) + ":= ");
- lc.getBindingExpr().accept(this, step + 1);
- }
+ @Override
+ public void visit(LetClause lc, Integer step) throws AsterixException {
+ out.print(skip(step) + "Let ");
+ lc.getVarExpr().accept(this, 0);
+ out.println(skip(step + 1) + ":= ");
+ lc.getBindingExpr().accept(this, step + 1);
+ }
- @Override
- public void visit(WhereClause wc, Integer step) throws AsterixException {
- out.println(skip(step) + "Where ");
- wc.getWhereExpr().accept(this, step + 1);
- }
+ @Override
+ public void visit(WhereClause wc, Integer step) throws AsterixException {
+ out.println(skip(step) + "Where ");
+ wc.getWhereExpr().accept(this, step + 1);
+ }
- @Override
- public void visit(OrderbyClause oc, Integer step) throws AsterixException {
- out.println(skip(step) + "Orderby");
- List<OrderModifier> mlist = oc.getModifierList();
- List<Expression> list = oc.getOrderbyList();
- for (int i = 0; i < list.size(); i++) {
- list.get(i).accept(this, step + 1);
- out.println(skip(step + 1) + mlist.get(i).toString());
- }
- out.println(skip(step));
- }
+ @Override
+ public void visit(OrderbyClause oc, Integer step) throws AsterixException {
+ out.println(skip(step) + "Orderby");
+ List<OrderModifier> mlist = oc.getModifierList();
+ List<Expression> list = oc.getOrderbyList();
+ for (int i = 0; i < list.size(); i++) {
+ list.get(i).accept(this, step + 1);
+ out.println(skip(step + 1) + mlist.get(i).toString());
+ }
+ out.println(skip(step));
+ }
- @Override
- public void visit(GroupbyClause gc, Integer step) throws AsterixException {
- out.println(skip(step) + "Groupby");
- for (GbyVariableExpressionPair pair : gc.getGbyPairList()) {
+ @Override
+ public void visit(GroupbyClause gc, Integer step) throws AsterixException {
+ out.println(skip(step) + "Groupby");
+ for (GbyVariableExpressionPair pair : gc.getGbyPairList()) {
- if (pair.getVar() != null) {
- pair.getVar().accept(this, step + 1);
- out.println(skip(step + 1) + ":=");
- }
+ if (pair.getVar() != null) {
+ pair.getVar().accept(this, step + 1);
+ out.println(skip(step + 1) + ":=");
+ }
- pair.getExpr().accept(this, step + 1);
- }
- if (!gc.getDecorPairList().isEmpty()) {
- out.println(skip(step + 1) + "Decor");
- for (GbyVariableExpressionPair pair : gc.getDecorPairList()) {
- if (pair.getVar() != null) {
- pair.getVar().accept(this, step + 1);
- out.println(skip(step + 1) + ":=");
- }
- pair.getExpr().accept(this, step + 1);
- }
- }
- out.println(skip(step + 1) + "With");
- for (VariableExpr exp : gc.getWithVarList()) {
- exp.accept(this, step + 1);
- }
- out.println(skip(step));
- }
+ pair.getExpr().accept(this, step + 1);
+ }
+ if (!gc.getDecorPairList().isEmpty()) {
+ out.println(skip(step + 1) + "Decor");
+ for (GbyVariableExpressionPair pair : gc.getDecorPairList()) {
+ if (pair.getVar() != null) {
+ pair.getVar().accept(this, step + 1);
+ out.println(skip(step + 1) + ":=");
+ }
+ pair.getExpr().accept(this, step + 1);
+ }
+ }
+ out.println(skip(step + 1) + "With");
+ for (VariableExpr exp : gc.getWithVarList()) {
+ exp.accept(this, step + 1);
+ }
+ out.println(skip(step));
+ }
- @Override
- public void visit(LimitClause lc, Integer step) throws AsterixException {
- out.println(skip(step) + "Limit");
- lc.getLimitExpr().accept(this, step + 1);
- if (lc.getOffset() != null) {
- out.println(skip(step + 1) + "Offset");
- lc.getOffset().accept(this, step + 2);
- }
- }
+ @Override
+ public void visit(LimitClause lc, Integer step) throws AsterixException {
+ out.println(skip(step) + "Limit");
+ lc.getLimitExpr().accept(this, step + 1);
+ if (lc.getOffset() != null) {
+ out.println(skip(step + 1) + "Offset");
+ lc.getOffset().accept(this, step + 2);
+ }
+ }
+
+ @Override
+ public void visit(FunctionDecl fd, Integer step) throws AsterixException {
+ out.println(skip(step) + "FunctionDecl " + fd.getSignature().getName()
+ + "(" + fd.getParamList().toString() + ") {");
+ fd.getFuncBody().accept(this, step + 1);
+ out.println(skip(step) + "}");
+ out.println();
+ }
+
+ @Override
+ public void visit(UnaryExpr u, Integer step) throws AsterixException {
+ if (u.getSign() != null) {
+ out.print(skip(step) + u.getSign() + " ");
+ u.getExpr().accept(this, 0);
+ } else
+ u.getExpr().accept(this, step);
+ }
+
+ @Override
+ public void visit(FieldAccessor fa, Integer step) throws AsterixException {
+ out.println(skip(step) + "FieldAccessor [");
+ fa.getExpr().accept(this, step + 1);
+ out.println(skip(step + 1) + "Field="
+ + ((FieldAccessor) fa).getIdent().getValue());
+ out.println(skip(step) + "]");
+
+ }
+
+ @Override
+ public void visit(IndexAccessor fa, Integer step) throws AsterixException {
+ out.println(skip(step) + "IndexAccessor [");
+ fa.getExpr().accept(this, step + 1);
+ out.print(skip(step + 1) + "Index: ");
+ out.println((((IndexAccessor) fa).isAny() ? "ANY"
+ : ((IndexAccessor) fa).getIndex()));
- @Override
- public void visit(FunctionDecl fd, Integer step) throws AsterixException {
- out.println(skip(step) + "FunctionDecl " + fd.getSignature().getName() + "(" + fd.getParamList().toString()
- + ") {");
- fd.getFuncBody().accept(this, step + 1);
- out.println(skip(step) + "}");
- out.println();
- }
+ out.println(skip(step) + "]");
- @Override
- public void visit(UnaryExpr u, Integer step) throws AsterixException {
- if (u.getSign() != null) {
- out.print(skip(step) + u.getSign() + " ");
- u.getExpr().accept(this, 0);
- } else
- u.getExpr().accept(this, step);
- }
+ }
- @Override
- public void visit(FieldAccessor fa, Integer step) throws AsterixException {
- out.println(skip(step) + "FieldAccessor [");
- fa.getExpr().accept(this, step + 1);
- out.println(skip(step + 1) + "Field=" + ((FieldAccessor) fa).getIdent().getValue());
- out.println(skip(step) + "]");
+ @Override
+ public void visit(TypeDecl t, Integer step) throws AsterixException {
+ out.println(skip(step) + "TypeDecl " + t.getIdent() + " [");
+ t.getTypeDef().accept(this, step + 1);
+ out.println(skip(step) + "]");
+ }
- }
+ @Override
+ public void visit(TypeReferenceExpression t, Integer arg)
+ throws AsterixException {
+ out.print(t.getIdent());
+ }
- @Override
- public void visit(IndexAccessor fa, Integer step) throws AsterixException {
- out.println(skip(step) + "IndexAccessor [");
- fa.getExpr().accept(this, step + 1);
- out.print(skip(step + 1) + "Index: ");
- out.println((((IndexAccessor) fa).isAny() ? "ANY" : ((IndexAccessor) fa).getIndex()));
+ @Override
+ public void visit(RecordTypeDefinition r, Integer step)
+ throws AsterixException {
+ if (r.getRecordKind() == RecordKind.CLOSED) {
+ out.print(skip(step) + "closed ");
+ } else {
+ out.print(skip(step) + "open ");
+ }
+ out.println("RecordType {");
+ Iterator<String> nameIter = r.getFieldNames().iterator();
+ Iterator<TypeExpression> typeIter = r.getFieldTypes().iterator();
+ Iterator<Boolean> isnullableIter = r.getNullableFields().iterator();
+ boolean first = true;
+ while (nameIter.hasNext()) {
+ if (first) {
+ first = false;
+ } else {
+ out.println(",");
+ }
+ String name = nameIter.next();
+ TypeExpression texp = typeIter.next();
+ Boolean isNullable = isnullableIter.next();
+ out.print(skip(step + 1) + name + " : ");
+ texp.accept(this, step + 2);
+ if (isNullable) {
+ out.print("?");
+ }
+ }
+ out.println();
+ out.println(skip(step) + "}");
+ }
- out.println(skip(step) + "]");
+ @Override
+ public void visit(OrderedListTypeDefinition x, Integer step)
+ throws AsterixException {
+ out.print("OrderedList [");
+ x.getItemTypeExpression().accept(this, step + 2);
+ out.println("]");
+ }
- }
+ @Override
+ public void visit(UnorderedListTypeDefinition x, Integer step)
+ throws AsterixException {
+ out.print("UnorderedList <");
+ x.getItemTypeExpression().accept(this, step + 2);
+ out.println(">");
+ }
- @Override
- public void visit(TypeDecl t, Integer step) throws AsterixException {
- out.println(skip(step) + "TypeDecl " + t.getIdent() + " [");
- t.getTypeDef().accept(this, step + 1);
- out.println(skip(step) + "]");
- }
+ @Override
+ public void visit(DatasetDecl dd, Integer step) throws AsterixException {
+ if (dd.getDatasetType() == DatasetType.INTERNAL) {
+ out.println(skip(step)
+ + "DatasetDecl"
+ + dd.getName()
+ + "("
+ + dd.getItemTypeName()
+ + ")"
+ + " partitioned by "
+ + ((InternalDetailsDecl) dd.getDatasetDetailsDecl())
+ .getPartitioningExprs());
+ } else if (dd.getDatasetType() == DatasetType.EXTERNAL) {
+ out.println(skip(step) + "DatasetDecl" + dd.getName() + "("
+ + dd.getItemTypeName() + ")" + "is an external dataset");
+ }
+ }
- @Override
- public void visit(TypeReferenceExpression t, Integer arg) throws AsterixException {
- out.print(t.getIdent());
- }
+ @Override
+ public void visit(DataverseDecl dv, Integer step) throws AsterixException {
+ out.println(skip(step) + "DataverseUse " + dv.getDataverseName());
+ }
- @Override
- public void visit(RecordTypeDefinition r, Integer step) throws AsterixException {
- if (r.getRecordKind() == RecordKind.CLOSED) {
- out.print(skip(step) + "closed ");
- } else {
- out.print(skip(step) + "open ");
- }
- out.println("RecordType {");
- Iterator<String> nameIter = r.getFieldNames().iterator();
- Iterator<TypeExpression> typeIter = r.getFieldTypes().iterator();
- Iterator<Boolean> isnullableIter = r.getNullableFields().iterator();
- boolean first = true;
- while (nameIter.hasNext()) {
- if (first) {
- first = false;
- } else {
- out.println(",");
- }
- String name = nameIter.next();
- TypeExpression texp = typeIter.next();
- Boolean isNullable = isnullableIter.next();
- out.print(skip(step + 1) + name + " : ");
- texp.accept(this, step + 2);
- if (isNullable) {
- out.print("?");
- }
- }
- out.println();
- out.println(skip(step) + "}");
- }
+ @Override
+ public void visit(NodegroupDecl ngd, Integer step) throws AsterixException {
+ out.println(skip(step) + "Nodegroup " + ngd.getNodeControllerNames());
+ }
- @Override
- public void visit(OrderedListTypeDefinition x, Integer step) throws AsterixException {
- out.print("OrderedList [");
- x.getItemTypeExpression().accept(this, step + 2);
- out.println("]");
- }
+ @Override
+ public void visit(LoadFromFileStatement stmtLoad, Integer arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
- @Override
- public void visit(UnorderedListTypeDefinition x, Integer step) throws AsterixException {
- out.print("UnorderedList <");
- x.getItemTypeExpression().accept(this, step + 2);
- out.println(">");
- }
+ }
- @Override
- public void visit(DatasetDecl dd, Integer step) throws AsterixException {
- if (dd.getDatasetType() == DatasetType.INTERNAL) {
- out.println(skip(step) + "DatasetDecl" + dd.getName() + "(" + dd.getItemTypeName() + ")"
- + " partitioned by " + ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getPartitioningExprs());
- } else if (dd.getDatasetType() == DatasetType.EXTERNAL) {
- out.println(skip(step) + "DatasetDecl" + dd.getName() + "(" + dd.getItemTypeName() + ")"
- + "is an external dataset");
- } else if (dd.getDatasetType() == DatasetType.FEED) {
- out.println(skip(step) + "DatasetDecl" + dd.getName() + "(" + dd.getItemTypeName() + ")"
- + "is an feed dataset");
- }
- }
+ @Override
+ public void visit(DropStatement stmtDel, Integer arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
- @Override
- public void visit(DataverseDecl dv, Integer step) throws AsterixException {
- out.println(skip(step) + "DataverseUse " + dv.getDataverseName());
- }
+ }
- @Override
- public void visit(NodegroupDecl ngd, Integer step) throws AsterixException {
- out.println(skip(step) + "Nodegroup " + ngd.getNodeControllerNames());
- }
+ @Override
+ public void visit(WriteStatement ws, Integer step) throws AsterixException {
+ out.print(skip(step) + "WriteOutputTo " + ws.getNcName() + ":"
+ + ws.getFileName());
+ if (ws.getWriterClassName() != null) {
+ out.print(" using " + ws.getWriterClassName());
+ }
+ out.println();
+ }
- @Override
- public void visit(LoadFromFileStatement stmtLoad, Integer arg) throws AsterixException {
- // TODO Auto-generated method stub
+ @Override
+ public void visit(SetStatement ss, Integer step) throws AsterixException {
+ out.println(skip(step) + "Set " + ss.getPropName() + "="
+ + ss.getPropValue());
+ }
- }
+ @Override
+ public void visit(CreateIndexStatement cis, Integer arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
- @Override
- public void visit(DropStatement stmtDel, Integer arg) throws AsterixException {
- // TODO Auto-generated method stub
+ }
- }
+ @Override
+ public void visit(DisconnectFeedStatement ss, Integer step)
+ throws AsterixException {
+ out.println(skip(step) + skip(step) + ss.getFeedName() + skip(step)
+ + ss.getDatasetName());
+ }
- @Override
- public void visit(WriteStatement ws, Integer step) throws AsterixException {
- out.print(skip(step) + "WriteOutputTo " + ws.getNcName() + ":" + ws.getFileName());
- if (ws.getWriterClassName() != null) {
- out.print(" using " + ws.getWriterClassName());
- }
- out.println();
- }
+ @Override
+ public void visit(UnionExpr u, Integer step) throws AsterixException {
+ out.println(skip(step) + "Union [");
+ for (Expression expr : u.getExprs()) {
+ expr.accept(this, step + 1);
+ }
+ out.println(skip(step) + "]");
+ }
- @Override
- public void visit(SetStatement ss, Integer step) throws AsterixException {
- out.println(skip(step) + "Set " + ss.getPropName() + "=" + ss.getPropValue());
- }
+ @Override
+ public void visit(DistinctClause dc, Integer step) throws AsterixException {
+ out.print(skip(step) + "Distinct ");
+ for (Expression expr : dc.getDistinctByExpr())
+ expr.accept(this, step + 1);
+ }
- @Override
- public void visit(CreateIndexStatement cis, Integer arg) throws AsterixException {
- // TODO Auto-generated method stub
+ @Override
+ public void visit(InsertStatement stmtDel, Integer arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
- }
+ }
- @Override
- public void visit(ControlFeedStatement ss, Integer step) throws AsterixException {
- out.println(skip(step) + ss.getOperationType() + skip(step) + ss.getDatasetName());
- }
+ @Override
+ public void visit(DeleteStatement stmtDel, Integer arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
- @Override
- public void visit(UnionExpr u, Integer step) throws AsterixException {
- out.println(skip(step) + "Union [");
- for (Expression expr : u.getExprs()) {
- expr.accept(this, step + 1);
- }
- out.println(skip(step) + "]");
- }
+ }
- @Override
- public void visit(DistinctClause dc, Integer step) throws AsterixException {
- out.print(skip(step) + "Distinct ");
- for (Expression expr : dc.getDistinctByExpr())
- expr.accept(this, step + 1);
- }
+ @Override
+ public void visit(UpdateStatement stmtDel, Integer arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
- @Override
- public void visit(InsertStatement stmtDel, Integer arg) throws AsterixException {
- // TODO Auto-generated method stub
+ }
- }
+ @Override
+ public void visit(UpdateClause updateClause, Integer arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
- @Override
- public void visit(DeleteStatement stmtDel, Integer arg) throws AsterixException {
- // TODO Auto-generated method stub
+ }
- }
+ @Override
+ public void visit(CreateDataverseStatement stmtDel, Integer arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ }
- @Override
- public void visit(UpdateStatement stmtDel, Integer arg) throws AsterixException {
- // TODO Auto-generated method stub
+ @Override
+ public void visit(IndexDropStatement stmtDel, Integer arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ }
- }
+ @Override
+ public void visit(NodeGroupDropStatement deleteStatement, Integer arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ }
- @Override
- public void visit(UpdateClause updateClause, Integer arg) throws AsterixException {
- // TODO Auto-generated method stub
+ @Override
+ public void visit(DataverseDropStatement deleteStatement, Integer arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ }
- }
+ @Override
+ public void visit(TypeDropStatement deleteStatement, Integer arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ }
- @Override
- public void visit(CreateDataverseStatement stmtDel, Integer arg) throws AsterixException {
- // TODO Auto-generated method stub
- }
+ @Override
+ public void visit(CreateFunctionStatement cfs, Integer arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
- @Override
- public void visit(IndexDropStatement stmtDel, Integer arg) throws AsterixException {
- // TODO Auto-generated method stub
- }
+ }
- @Override
- public void visit(NodeGroupDropStatement deleteStatement, Integer arg) throws AsterixException {
- // TODO Auto-generated method stub
- }
+ @Override
+ public void visit(FunctionDropStatement fds, Integer arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
- @Override
- public void visit(DataverseDropStatement deleteStatement, Integer arg) throws AsterixException {
- // TODO Auto-generated method stub
- }
+ }
- @Override
- public void visit(TypeDropStatement deleteStatement, Integer arg) throws AsterixException {
- // TODO Auto-generated method stub
- }
+ @Override
+ public void visit(CreateFeedStatement stmtDel, Integer arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
- @Override
- public void visit(CreateFunctionStatement cfs, Integer arg) throws AsterixException {
- // TODO Auto-generated method stub
+ }
- }
+ @Override
+ public void visit(ConnectFeedStatement stmtDel, Integer arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
- @Override
- public void visit(FunctionDropStatement fds, Integer arg) throws AsterixException {
- // TODO Auto-generated method stub
+ }
- }
+ @Override
+ public void visit(FeedDropStatement stmt, Integer arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
- @Override
- public void visit(BeginFeedStatement stmtDel, Integer arg) throws AsterixException {
- // TODO Auto-generated method stub
+ }
- }
+ @Override
+ public void visit(CompactStatement fds, Integer arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
- @Override
- public void visit(CompactStatement fds, Integer arg) throws AsterixException {
- // TODO Auto-generated method stub
-
- }
+ }
}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/IAqlExpressionVisitor.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/IAqlExpressionVisitor.java
index d9a92b4..4077cb6 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/IAqlExpressionVisitor.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/IAqlExpressionVisitor.java
@@ -14,18 +14,20 @@
*/
package edu.uci.ics.asterix.aql.expression.visitor;
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
import edu.uci.ics.asterix.aql.expression.CallExpr;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.CompactStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
import edu.uci.ics.asterix.aql.expression.DatasetDecl;
import edu.uci.ics.asterix.aql.expression.DataverseDecl;
import edu.uci.ics.asterix.aql.expression.DataverseDropStatement;
import edu.uci.ics.asterix.aql.expression.DeleteStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.DistinctClause;
+import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
import edu.uci.ics.asterix.aql.expression.DropStatement;
import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
import edu.uci.ics.asterix.aql.expression.FieldAccessor;
@@ -67,102 +69,128 @@
public interface IAqlExpressionVisitor<R, T> {
- R visitQuery(Query q, T arg) throws AsterixException;
+ R visitQuery(Query q, T arg) throws AsterixException;
- R visitFunctionDecl(FunctionDecl fd, T arg) throws AsterixException;
+ R visitFunctionDecl(FunctionDecl fd, T arg) throws AsterixException;
- R visitTypeDecl(TypeDecl td, T arg) throws AsterixException;
+ R visitTypeDecl(TypeDecl td, T arg) throws AsterixException;
- R visitNodegroupDecl(NodegroupDecl ngd, T arg) throws AsterixException;
+ R visitNodegroupDecl(NodegroupDecl ngd, T arg) throws AsterixException;
- R visitDatasetDecl(DatasetDecl dd, T arg) throws AsterixException;
+ R visitDatasetDecl(DatasetDecl dd, T arg) throws AsterixException;
- R visitLoadFromFileStatement(LoadFromFileStatement stmtLoad, T arg) throws AsterixException;
+ R visitLoadFromFileStatement(LoadFromFileStatement stmtLoad, T arg)
+ throws AsterixException;
- R visitDropStatement(DropStatement del, T arg) throws AsterixException;
+ R visitDropStatement(DropStatement del, T arg) throws AsterixException;
- R visitInsertStatement(InsertStatement insert, T arg) throws AsterixException;
+ R visitInsertStatement(InsertStatement insert, T arg)
+ throws AsterixException;
- R visitDeleteStatement(DeleteStatement del, T arg) throws AsterixException;
+ R visitDeleteStatement(DeleteStatement del, T arg) throws AsterixException;
- R visitUpdateStatement(UpdateStatement update, T arg) throws AsterixException;
+ R visitUpdateStatement(UpdateStatement update, T arg)
+ throws AsterixException;
- R visitUpdateClause(UpdateClause del, T arg) throws AsterixException;
+ R visitUpdateClause(UpdateClause del, T arg) throws AsterixException;
- R visitTypeReferenceExpression(TypeReferenceExpression tre, T arg) throws AsterixException;
+ R visitTypeReferenceExpression(TypeReferenceExpression tre, T arg)
+ throws AsterixException;
- R visitRecordTypeDefiniton(RecordTypeDefinition tre, T arg) throws AsterixException;
+ R visitRecordTypeDefiniton(RecordTypeDefinition tre, T arg)
+ throws AsterixException;
- R visitOrderedListTypeDefiniton(OrderedListTypeDefinition olte, T arg) throws AsterixException;
+ R visitOrderedListTypeDefiniton(OrderedListTypeDefinition olte, T arg)
+ throws AsterixException;
- R visitUnorderedListTypeDefiniton(UnorderedListTypeDefinition ulte, T arg) throws AsterixException;
+ R visitUnorderedListTypeDefiniton(UnorderedListTypeDefinition ulte, T arg)
+ throws AsterixException;
- R visitLiteralExpr(LiteralExpr l, T arg) throws AsterixException;
+ R visitLiteralExpr(LiteralExpr l, T arg) throws AsterixException;
- R visitVariableExpr(VariableExpr v, T arg) throws AsterixException;
+ R visitVariableExpr(VariableExpr v, T arg) throws AsterixException;
- R visitListConstructor(ListConstructor lc, T arg) throws AsterixException;
+ R visitListConstructor(ListConstructor lc, T arg) throws AsterixException;
- R visitRecordConstructor(RecordConstructor rc, T arg) throws AsterixException;
+ R visitRecordConstructor(RecordConstructor rc, T arg)
+ throws AsterixException;
- R visitOperatorExpr(OperatorExpr ifbo, T arg) throws AsterixException;
+ R visitOperatorExpr(OperatorExpr ifbo, T arg) throws AsterixException;
- R visitFieldAccessor(FieldAccessor fa, T arg) throws AsterixException;
+ R visitFieldAccessor(FieldAccessor fa, T arg) throws AsterixException;
- R visitIndexAccessor(IndexAccessor ia, T arg) throws AsterixException;
+ R visitIndexAccessor(IndexAccessor ia, T arg) throws AsterixException;
- R visitIfExpr(IfExpr ifexpr, T arg) throws AsterixException;
+ R visitIfExpr(IfExpr ifexpr, T arg) throws AsterixException;
- R visitFlworExpression(FLWOGRExpression flwor, T arg) throws AsterixException;
+ R visitFlworExpression(FLWOGRExpression flwor, T arg)
+ throws AsterixException;
- R visitQuantifiedExpression(QuantifiedExpression qe, T arg) throws AsterixException;
+ R visitQuantifiedExpression(QuantifiedExpression qe, T arg)
+ throws AsterixException;
- R visitForClause(ForClause fc, T arg) throws AsterixException;
+ R visitForClause(ForClause fc, T arg) throws AsterixException;
- R visitLetClause(LetClause lc, T arg) throws AsterixException;
+ R visitLetClause(LetClause lc, T arg) throws AsterixException;
- R visitWhereClause(WhereClause wc, T arg) throws AsterixException;
+ R visitWhereClause(WhereClause wc, T arg) throws AsterixException;
- R visitOrderbyClause(OrderbyClause oc, T arg) throws AsterixException;
+ R visitOrderbyClause(OrderbyClause oc, T arg) throws AsterixException;
- R visitGroupbyClause(GroupbyClause gc, T arg) throws AsterixException;
+ R visitGroupbyClause(GroupbyClause gc, T arg) throws AsterixException;
- R visitLimitClause(LimitClause lc, T arg) throws AsterixException;
+ R visitLimitClause(LimitClause lc, T arg) throws AsterixException;
- R visitDistinctClause(DistinctClause dc, T arg) throws AsterixException;
+ R visitDistinctClause(DistinctClause dc, T arg) throws AsterixException;
- R visitUnaryExpr(UnaryExpr u, T arg) throws AsterixException;
+ R visitUnaryExpr(UnaryExpr u, T arg) throws AsterixException;
- R visitUnionExpr(UnionExpr u, T arg) throws AsterixException;
+ R visitUnionExpr(UnionExpr u, T arg) throws AsterixException;
- R visitCreateIndexStatement(CreateIndexStatement cis, T arg) throws AsterixException;
+ R visitCreateIndexStatement(CreateIndexStatement cis, T arg)
+ throws AsterixException;
- R visitCreateDataverseStatement(CreateDataverseStatement del, T arg) throws AsterixException;
+ R visitCreateDataverseStatement(CreateDataverseStatement del, T arg)
+ throws AsterixException;
- R visitIndexDropStatement(IndexDropStatement del, T arg) throws AsterixException;
+ R visitIndexDropStatement(IndexDropStatement del, T arg)
+ throws AsterixException;
- R visitNodeGroupDropStatement(NodeGroupDropStatement del, T arg) throws AsterixException;
+ R visitNodeGroupDropStatement(NodeGroupDropStatement del, T arg)
+ throws AsterixException;
- R visitDataverseDropStatement(DataverseDropStatement del, T arg) throws AsterixException;
+ R visitDataverseDropStatement(DataverseDropStatement del, T arg)
+ throws AsterixException;
- R visitTypeDropStatement(TypeDropStatement del, T arg) throws AsterixException;
+ R visitTypeDropStatement(TypeDropStatement del, T arg)
+ throws AsterixException;
- R visitWriteStatement(WriteStatement ws, T arg) throws AsterixException;
+ R visitWriteStatement(WriteStatement ws, T arg) throws AsterixException;
- R visitSetStatement(SetStatement ss, T arg) throws AsterixException;
+ R visitSetStatement(SetStatement ss, T arg) throws AsterixException;
- R visitBeginFeedStatement(BeginFeedStatement bf, T arg) throws AsterixException;
+ R visitDisconnectFeedStatement(DisconnectFeedStatement del, T arg)
+ throws AsterixException;
- R visitControlFeedStatement(ControlFeedStatement del, T arg) throws AsterixException;
+ R visitConnectFeedStatement(ConnectFeedStatement del, T arg)
+ throws AsterixException;
- R visitCallExpr(CallExpr pf, T arg) throws AsterixException;
+ R visitCreateFeedStatement(CreateFeedStatement del, T arg)
+ throws AsterixException;
- R visitDataverseDecl(DataverseDecl dv, T arg) throws AsterixException;
+ R visitDropFeedStatement(FeedDropStatement del, T arg)
+ throws AsterixException;
- R visit(CreateFunctionStatement cfs, T arg) throws AsterixException;
+ R visitCallExpr(CallExpr pf, T arg) throws AsterixException;
- R visitFunctionDropStatement(FunctionDropStatement del, T arg) throws AsterixException;
+ R visitDataverseDecl(DataverseDecl dv, T arg) throws AsterixException;
- R visitCompactStatement(CompactStatement del, T arg) throws AsterixException;
+ R visit(CreateFunctionStatement cfs, T arg) throws AsterixException;
+
+ R visitFunctionDropStatement(FunctionDropStatement del, T arg)
+ throws AsterixException;
+
+ R visitCompactStatement(CompactStatement del, T arg)
+ throws AsterixException;
}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/IAqlVisitorWithVoidReturn.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/IAqlVisitorWithVoidReturn.java
index 61a7183..79653c2 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/IAqlVisitorWithVoidReturn.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/IAqlVisitorWithVoidReturn.java
@@ -14,18 +14,20 @@
*/
package edu.uci.ics.asterix.aql.expression.visitor;
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
import edu.uci.ics.asterix.aql.expression.CallExpr;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.CompactStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
import edu.uci.ics.asterix.aql.expression.DatasetDecl;
import edu.uci.ics.asterix.aql.expression.DataverseDecl;
import edu.uci.ics.asterix.aql.expression.DataverseDropStatement;
import edu.uci.ics.asterix.aql.expression.DeleteStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.DistinctClause;
+import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
import edu.uci.ics.asterix.aql.expression.DropStatement;
import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
import edu.uci.ics.asterix.aql.expression.FieldAccessor;
@@ -155,9 +157,13 @@
void visit(TypeDropStatement stmtDel, T arg) throws AsterixException;
- void visit(BeginFeedStatement stmtDel, T arg) throws AsterixException;
+ void visit(DisconnectFeedStatement stmtDel, T arg) throws AsterixException;
- void visit(ControlFeedStatement stmtDel, T arg) throws AsterixException;
+ void visit(ConnectFeedStatement stmtDel, T arg) throws AsterixException;
+
+ void visit(CreateFeedStatement stmt, T arg) throws AsterixException;
+
+ void visit(FeedDropStatement stmt, T arg) throws AsterixException;
void visit(CreateFunctionStatement cfs, T arg) throws AsterixException;
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java
index 203758d..d0a4ed4 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java
@@ -24,11 +24,12 @@
import edu.uci.ics.asterix.aql.base.Clause;
import edu.uci.ics.asterix.aql.base.Expression;
import edu.uci.ics.asterix.aql.base.Expression.Kind;
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
import edu.uci.ics.asterix.aql.expression.CallExpr;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.CompactStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
import edu.uci.ics.asterix.aql.expression.DatasetDecl;
@@ -38,6 +39,7 @@
import edu.uci.ics.asterix.aql.expression.DistinctClause;
import edu.uci.ics.asterix.aql.expression.DropStatement;
import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
+import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
import edu.uci.ics.asterix.aql.expression.FieldAccessor;
import edu.uci.ics.asterix.aql.expression.FieldBinding;
import edu.uci.ics.asterix.aql.expression.ForClause;
@@ -92,467 +94,555 @@
public final class AqlRewriter {
- private final Query topExpr;
- private final List<FunctionDecl> declaredFunctions;
- private final AqlRewritingContext context;
- private final MetadataTransactionContext mdTxnCtx;
- private final AqlMetadataProvider metadataProvider;
+ private final Query topExpr;
+ private final List<FunctionDecl> declaredFunctions;
+ private final AqlRewritingContext context;
+ private final MetadataTransactionContext mdTxnCtx;
+ private final AqlMetadataProvider metadataProvider;
- private enum DfsColor {
- WHITE,
- GRAY,
- BLACK
- }
+ private enum DfsColor {
+ WHITE, GRAY, BLACK
+ }
- public AqlRewriter(List<FunctionDecl> declaredFunctions, Query topExpr, AqlMetadataProvider metadataProvider) {
- this.topExpr = topExpr;
- context = new AqlRewritingContext(topExpr.getVarCounter());
- this.declaredFunctions = declaredFunctions;
- this.mdTxnCtx = metadataProvider.getMetadataTxnContext();
- this.metadataProvider = metadataProvider;
- }
+ public AqlRewriter(List<FunctionDecl> declaredFunctions, Query topExpr,
+ AqlMetadataProvider metadataProvider) {
+ this.topExpr = topExpr;
+ context = new AqlRewritingContext(topExpr.getVarCounter());
+ this.declaredFunctions = declaredFunctions;
+ this.mdTxnCtx = metadataProvider.getMetadataTxnContext();
+ this.metadataProvider = metadataProvider;
+ }
- public Query getExpr() {
- return topExpr;
- }
+ public Query getExpr() {
+ return topExpr;
+ }
- public int getVarCounter() {
- return context.getVarCounter();
- }
+ public int getVarCounter() {
+ return context.getVarCounter();
+ }
- public void rewrite() throws AsterixException {
- wrapInLets();
- inlineDeclaredUdfs();
- }
+ public void rewrite() throws AsterixException {
+ wrapInLets();
+ inlineDeclaredUdfs();
+ }
- private void wrapInLets() {
- // If the top expression of the main statement is not a FLWOR, it wraps
- // it into a let clause.
- if (topExpr == null) {
- return;
- }
- Expression body = topExpr.getBody();
- if (body.getKind() != Kind.FLWOGR_EXPRESSION) {
- VarIdentifier var = context.newVariable();
- VariableExpr v = new VariableExpr(var);
- LetClause c1 = new LetClause(v, body);
- ArrayList<Clause> clauseList = new ArrayList<Clause>(1);
- clauseList.add(c1);
- FLWOGRExpression newBody = new FLWOGRExpression(clauseList, new VariableExpr(var));
- topExpr.setBody(newBody);
- }
- }
+ private void wrapInLets() {
+ // If the top expression of the main statement is not a FLWOR, it wraps
+ // it into a let clause.
+ if (topExpr == null) {
+ return;
+ }
+ Expression body = topExpr.getBody();
+ if (body.getKind() != Kind.FLWOGR_EXPRESSION) {
+ VarIdentifier var = context.newVariable();
+ VariableExpr v = new VariableExpr(var);
+ LetClause c1 = new LetClause(v, body);
+ ArrayList<Clause> clauseList = new ArrayList<Clause>(1);
+ clauseList.add(c1);
+ FLWOGRExpression newBody = new FLWOGRExpression(clauseList,
+ new VariableExpr(var));
+ topExpr.setBody(newBody);
+ }
+ }
- private void inlineDeclaredUdfs() throws AsterixException {
- if (topExpr == null) {
- return;
- }
- List<FunctionSignature> funIds = new ArrayList<FunctionSignature>();
- for (FunctionDecl fdecl : declaredFunctions) {
- funIds.add(fdecl.getSignature());
- }
+ private void inlineDeclaredUdfs() throws AsterixException {
+ if (topExpr == null) {
+ return;
+ }
+ List<FunctionSignature> funIds = new ArrayList<FunctionSignature>();
+ for (FunctionDecl fdecl : declaredFunctions) {
+ funIds.add(fdecl.getSignature());
+ }
- List<FunctionDecl> otherFDecls = new ArrayList<FunctionDecl>();
- buildOtherUdfs(topExpr.getBody(), otherFDecls, funIds);
- declaredFunctions.addAll(otherFDecls);
- if (!declaredFunctions.isEmpty()) {
- InlineUdfsVisitor visitor = new InlineUdfsVisitor(context);
- while (topExpr.accept(visitor, declaredFunctions)) {
- // loop until no more changes
- }
- }
- declaredFunctions.removeAll(otherFDecls);
- }
+ List<FunctionDecl> otherFDecls = new ArrayList<FunctionDecl>();
+ buildOtherUdfs(topExpr.getBody(), otherFDecls, funIds);
+ declaredFunctions.addAll(otherFDecls);
+ if (!declaredFunctions.isEmpty()) {
+ InlineUdfsVisitor visitor = new InlineUdfsVisitor(context);
+ while (topExpr.accept(visitor, declaredFunctions)) {
+ // loop until no more changes
+ }
+ }
+ declaredFunctions.removeAll(otherFDecls);
+ }
- private void buildOtherUdfs(Expression expression, List<FunctionDecl> functionDecls,
- List<FunctionSignature> declaredFunctions) throws AsterixException {
- if (expression == null) {
- return;
- }
- String value = metadataProvider.getConfig().get(FunctionUtils.IMPORT_PRIVATE_FUNCTIONS);
- boolean includePrivateFunctions = (value != null) ? Boolean.valueOf(value.toLowerCase()) : false;
- Set<FunctionSignature> functionCalls = getFunctionCalls(expression);
- for (FunctionSignature signature : functionCalls) {
+ private void buildOtherUdfs(Expression expression,
+ List<FunctionDecl> functionDecls,
+ List<FunctionSignature> declaredFunctions) throws AsterixException {
+ if (expression == null) {
+ return;
+ }
+ String value = metadataProvider.getConfig().get(
+ FunctionUtils.IMPORT_PRIVATE_FUNCTIONS);
+ boolean includePrivateFunctions = (value != null) ? Boolean
+ .valueOf(value.toLowerCase()) : false;
+ Set<FunctionSignature> functionCalls = getFunctionCalls(expression);
+ for (FunctionSignature signature : functionCalls) {
- if (declaredFunctions != null && declaredFunctions.contains(signature)) {
- continue;
- }
+ if (declaredFunctions != null
+ && declaredFunctions.contains(signature)) {
+ continue;
+ }
- FunctionDecl functionDecl = lookupUserDefinedFunctionDecl(signature);
- if (functionDecl != null) {
- if (functionDecls.contains(functionDecl)) {
- throw new AsterixException(" Detected recursvity!");
- } else {
- functionDecls.add(functionDecl);
- buildOtherUdfs(functionDecl.getFuncBody(), functionDecls, declaredFunctions);
- }
- } else {
- if (AsterixBuiltinFunctions.isBuiltinCompilerFunction(signature, includePrivateFunctions)) {
- continue;
- } else {
- throw new AsterixException(" unknown function " + signature);
- }
- }
- }
- }
+ Function function = lookupUserDefinedFunctionDecl(signature);
+ if (function == null) {
+ if (AsterixBuiltinFunctions.isBuiltinCompilerFunction(
+ signature, includePrivateFunctions)) {
+ continue;
+ }
+ StringBuilder messageBuilder = new StringBuilder();
+ if (functionDecls.size() > 0) {
+ messageBuilder.append(" function "
+ + functionDecls.get(functionDecls.size() - 1)
+ .getSignature() + " depends upon function "
+ + signature + " which is undefined");
+ } else {
+ messageBuilder.append(" function " + signature
+ + " is undefined ");
+ }
+ throw new AsterixException(messageBuilder.toString());
+ }
- private FunctionDecl lookupUserDefinedFunctionDecl(FunctionSignature signature) throws AsterixException {
- if (signature.getNamespace() == null) {
- return null;
- }
- Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, signature);
- if (function == null) {
- return null;
- }
- return FunctionUtils.getFunctionDecl(function);
+ if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)) {
+ FunctionDecl functionDecl = FunctionUtils
+ .getFunctionDecl(function);
+ if (functionDecl != null) {
+ if (functionDecls.contains(functionDecl)) {
+ throw new AsterixException(
+ "ERROR:Recursive invocation "
+ + functionDecls.get(
+ functionDecls.size() - 1)
+ .getSignature() + " <==> "
+ + functionDecl.getSignature());
+ }
+ functionDecls.add(functionDecl);
+ buildOtherUdfs(functionDecl.getFuncBody(), functionDecls,
+ declaredFunctions);
+ }
+ }
+ }
- }
+ }
- private Set<FunctionSignature> getFunctionCalls(Expression expression) throws AsterixException {
- Map<AsterixFunction, DfsColor> color = new HashMap<AsterixFunction, DfsColor>();
- Map<AsterixFunction, List<AsterixFunction>> arcs = new HashMap<AsterixFunction, List<AsterixFunction>>();
- GatherFunctionCalls gfc = new GatherFunctionCalls();
- expression.accept(gfc, null);
- return gfc.getCalls();
- }
+ private Function lookupUserDefinedFunctionDecl(FunctionSignature signature)
+ throws AsterixException {
+ if (signature.getNamespace() == null) {
+ return null;
+ }
+ return MetadataManager.INSTANCE.getFunction(mdTxnCtx, signature);
+ }
- private static class GatherFunctionCalls implements IAqlExpressionVisitor<Void, Void> {
+ private Set<FunctionSignature> getFunctionCalls(Expression expression)
+ throws AsterixException {
+ Map<AsterixFunction, DfsColor> color = new HashMap<AsterixFunction, DfsColor>();
+ Map<AsterixFunction, List<AsterixFunction>> arcs = new HashMap<AsterixFunction, List<AsterixFunction>>();
+ GatherFunctionCalls gfc = new GatherFunctionCalls();
+ expression.accept(gfc, null);
+ return gfc.getCalls();
+ }
- private final Set<FunctionSignature> calls = new HashSet<FunctionSignature>();
+ private static class GatherFunctionCalls implements
+ IAqlExpressionVisitor<Void, Void> {
- public GatherFunctionCalls() {
- }
+ private final Set<FunctionSignature> calls = new HashSet<FunctionSignature>();
- @Override
- public Void visitCallExpr(CallExpr pf, Void arg) throws AsterixException {
- calls.add(pf.getFunctionSignature());
- for (Expression e : pf.getExprList()) {
- e.accept(this, arg);
- }
- return null;
- }
+ public GatherFunctionCalls() {
+ }
- @Override
- public Void visitCreateIndexStatement(CreateIndexStatement cis, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitCallExpr(CallExpr pf, Void arg)
+ throws AsterixException {
+ calls.add(pf.getFunctionSignature());
+ for (Expression e : pf.getExprList()) {
+ e.accept(this, arg);
+ }
+ return null;
+ }
- @Override
- public Void visitDataverseDecl(DataverseDecl dv, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitCreateIndexStatement(CreateIndexStatement cis, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitDeleteStatement(DeleteStatement del, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitDataverseDecl(DataverseDecl dv, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitDistinctClause(DistinctClause dc, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitDeleteStatement(DeleteStatement del, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitDropStatement(DropStatement del, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitDistinctClause(DistinctClause dc, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitDatasetDecl(DatasetDecl dd, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitDropStatement(DropStatement del, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitFieldAccessor(FieldAccessor fa, Void arg) throws AsterixException {
- fa.getExpr().accept(this, arg);
- return null;
- }
+ @Override
+ public Void visitDatasetDecl(DatasetDecl dd, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitFlworExpression(FLWOGRExpression flwor, Void arg) throws AsterixException {
- for (Clause c : flwor.getClauseList()) {
- c.accept(this, arg);
- }
- flwor.getReturnExpr().accept(this, arg);
- return null;
- }
+ @Override
+ public Void visitFieldAccessor(FieldAccessor fa, Void arg)
+ throws AsterixException {
+ fa.getExpr().accept(this, arg);
+ return null;
+ }
- @Override
- public Void visitForClause(ForClause fc, Void arg) throws AsterixException {
- fc.getInExpr().accept(this, arg);
- if (fc.getPosVarExpr() != null) {
- fc.getPosVarExpr().accept(this, arg);
- }
- return null;
- }
+ @Override
+ public Void visitFlworExpression(FLWOGRExpression flwor, Void arg)
+ throws AsterixException {
+ for (Clause c : flwor.getClauseList()) {
+ c.accept(this, arg);
+ }
+ flwor.getReturnExpr().accept(this, arg);
+ return null;
+ }
- @Override
- public Void visitFunctionDecl(FunctionDecl fd, Void arg) throws AsterixException {
- fd.getFuncBody().accept(this, arg);
- return null;
- }
+ @Override
+ public Void visitForClause(ForClause fc, Void arg)
+ throws AsterixException {
+ fc.getInExpr().accept(this, arg);
+ if (fc.getPosVarExpr() != null) {
+ fc.getPosVarExpr().accept(this, arg);
+ }
+ return null;
+ }
- @Override
- public Void visitGroupbyClause(GroupbyClause gc, Void arg) throws AsterixException {
- for (GbyVariableExpressionPair p : gc.getGbyPairList()) {
- p.getExpr().accept(this, arg);
- }
- for (GbyVariableExpressionPair p : gc.getDecorPairList()) {
- p.getExpr().accept(this, arg);
- }
- return null;
- }
+ @Override
+ public Void visitFunctionDecl(FunctionDecl fd, Void arg)
+ throws AsterixException {
+ fd.getFuncBody().accept(this, arg);
+ return null;
+ }
- @Override
- public Void visitIfExpr(IfExpr ifexpr, Void arg) throws AsterixException {
- ifexpr.getCondExpr().accept(this, arg);
- ifexpr.getThenExpr().accept(this, arg);
- ifexpr.getElseExpr().accept(this, arg);
- return null;
- }
+ @Override
+ public Void visitGroupbyClause(GroupbyClause gc, Void arg)
+ throws AsterixException {
+ for (GbyVariableExpressionPair p : gc.getGbyPairList()) {
+ p.getExpr().accept(this, arg);
+ }
+ for (GbyVariableExpressionPair p : gc.getDecorPairList()) {
+ p.getExpr().accept(this, arg);
+ }
+ return null;
+ }
- @Override
- public Void visitIndexAccessor(IndexAccessor ia, Void arg) throws AsterixException {
- ia.getExpr().accept(this, arg);
- return null;
- }
+ @Override
+ public Void visitIfExpr(IfExpr ifexpr, Void arg)
+ throws AsterixException {
+ ifexpr.getCondExpr().accept(this, arg);
+ ifexpr.getThenExpr().accept(this, arg);
+ ifexpr.getElseExpr().accept(this, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitIndexAccessor(IndexAccessor ia, Void arg)
+ throws AsterixException {
+ ia.getExpr().accept(this, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitInsertStatement(InsertStatement insert, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitLetClause(LetClause lc, Void arg)
+ throws AsterixException {
+ lc.getBindingExpr().accept(this, arg);
+ return null;
+ }
- @Override
- public Void visitInsertStatement(InsertStatement insert, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitLimitClause(LimitClause lc, Void arg)
+ throws AsterixException {
+ lc.getLimitExpr().accept(this, arg);
+ if (lc.getOffset() != null) {
+ lc.getOffset().accept(this, arg);
+ }
+ return null;
+ }
- @Override
- public Void visitLetClause(LetClause lc, Void arg) throws AsterixException {
- lc.getBindingExpr().accept(this, arg);
- return null;
- }
+ @Override
+ public Void visitListConstructor(ListConstructor lc, Void arg)
+ throws AsterixException {
+ for (Expression e : lc.getExprList()) {
+ e.accept(this, arg);
+ }
+ return null;
+ }
- @Override
- public Void visitLimitClause(LimitClause lc, Void arg) throws AsterixException {
- lc.getLimitExpr().accept(this, arg);
- if (lc.getOffset() != null) {
- lc.getOffset().accept(this, arg);
- }
- return null;
- }
+ @Override
+ public Void visitLiteralExpr(LiteralExpr l, Void arg)
+ throws AsterixException {
+ // do nothing
+ return null;
+ }
- @Override
- public Void visitListConstructor(ListConstructor lc, Void arg) throws AsterixException {
- for (Expression e : lc.getExprList()) {
- e.accept(this, arg);
- }
- return null;
- }
+ @Override
+ public Void visitLoadFromFileStatement(LoadFromFileStatement stmtLoad,
+ Void arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitLiteralExpr(LiteralExpr l, Void arg) throws AsterixException {
- // do nothing
- return null;
- }
+ @Override
+ public Void visitNodegroupDecl(NodegroupDecl ngd, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitLoadFromFileStatement(LoadFromFileStatement stmtLoad, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitOperatorExpr(OperatorExpr op, Void arg)
+ throws AsterixException {
+ for (Expression e : op.getExprList()) {
+ e.accept(this, arg);
+ }
+ return null;
+ }
- @Override
- public Void visitNodegroupDecl(NodegroupDecl ngd, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitOrderbyClause(OrderbyClause oc, Void arg)
+ throws AsterixException {
+ for (Expression e : oc.getOrderbyList()) {
+ e.accept(this, arg);
+ }
+ return null;
+ }
- @Override
- public Void visitOperatorExpr(OperatorExpr op, Void arg) throws AsterixException {
- for (Expression e : op.getExprList()) {
- e.accept(this, arg);
- }
- return null;
- }
+ @Override
+ public Void visitOrderedListTypeDefiniton(
+ OrderedListTypeDefinition olte, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitOrderbyClause(OrderbyClause oc, Void arg) throws AsterixException {
- for (Expression e : oc.getOrderbyList()) {
- e.accept(this, arg);
- }
- return null;
- }
+ @Override
+ public Void visitQuantifiedExpression(QuantifiedExpression qe, Void arg)
+ throws AsterixException {
+ for (QuantifiedPair qp : qe.getQuantifiedList()) {
+ qp.getExpr().accept(this, arg);
+ }
+ qe.getSatisfiesExpr().accept(this, arg);
+ return null;
+ }
- @Override
- public Void visitOrderedListTypeDefiniton(OrderedListTypeDefinition olte, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitQuery(Query q, Void arg) throws AsterixException {
+ q.getBody().accept(this, arg);
+ return null;
+ }
- @Override
- public Void visitQuantifiedExpression(QuantifiedExpression qe, Void arg) throws AsterixException {
- for (QuantifiedPair qp : qe.getQuantifiedList()) {
- qp.getExpr().accept(this, arg);
- }
- qe.getSatisfiesExpr().accept(this, arg);
- return null;
- }
+ @Override
+ public Void visitRecordConstructor(RecordConstructor rc, Void arg)
+ throws AsterixException {
+ for (FieldBinding fb : rc.getFbList()) {
+ fb.getLeftExpr().accept(this, arg);
+ fb.getRightExpr().accept(this, arg);
+ }
+ return null;
+ }
- @Override
- public Void visitQuery(Query q, Void arg) throws AsterixException {
- q.getBody().accept(this, arg);
- return null;
- }
+ @Override
+ public Void visitRecordTypeDefiniton(RecordTypeDefinition tre, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitRecordConstructor(RecordConstructor rc, Void arg) throws AsterixException {
- for (FieldBinding fb : rc.getFbList()) {
- fb.getLeftExpr().accept(this, arg);
- fb.getRightExpr().accept(this, arg);
- }
- return null;
- }
+ @Override
+ public Void visitSetStatement(SetStatement ss, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitRecordTypeDefiniton(RecordTypeDefinition tre, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitTypeDecl(TypeDecl td, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitSetStatement(SetStatement ss, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitTypeReferenceExpression(TypeReferenceExpression tre,
+ Void arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitTypeDecl(TypeDecl td, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitUnaryExpr(UnaryExpr u, Void arg)
+ throws AsterixException {
+ u.getExpr().accept(this, arg);
+ return null;
+ }
- @Override
- public Void visitTypeReferenceExpression(TypeReferenceExpression tre, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitUnionExpr(UnionExpr u, Void arg)
+ throws AsterixException {
+ for (Expression e : u.getExprs()) {
+ e.accept(this, arg);
+ }
+ return null;
+ }
- @Override
- public Void visitUnaryExpr(UnaryExpr u, Void arg) throws AsterixException {
- u.getExpr().accept(this, arg);
- return null;
- }
+ @Override
+ public Void visitUnorderedListTypeDefiniton(
+ UnorderedListTypeDefinition ulte, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitUnionExpr(UnionExpr u, Void arg) throws AsterixException {
- for (Expression e : u.getExprs()) {
- e.accept(this, arg);
- }
- return null;
- }
+ @Override
+ public Void visitUpdateClause(UpdateClause del, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitUnorderedListTypeDefiniton(UnorderedListTypeDefinition ulte, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitUpdateStatement(UpdateStatement update, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitUpdateClause(UpdateClause del, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitVariableExpr(VariableExpr v, Void arg)
+ throws AsterixException {
+ // do nothing
+ return null;
+ }
- @Override
- public Void visitUpdateStatement(UpdateStatement update, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitWhereClause(WhereClause wc, Void arg)
+ throws AsterixException {
+ wc.getWhereExpr().accept(this, arg);
+ return null;
+ }
- @Override
- public Void visitVariableExpr(VariableExpr v, Void arg) throws AsterixException {
- // do nothing
- return null;
- }
+ @Override
+ public Void visitWriteStatement(WriteStatement ws, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitWhereClause(WhereClause wc, Void arg) throws AsterixException {
- wc.getWhereExpr().accept(this, arg);
- return null;
- }
+ public Set<FunctionSignature> getCalls() {
+ return calls;
+ }
- @Override
- public Void visitWriteStatement(WriteStatement ws, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitCreateDataverseStatement(CreateDataverseStatement del,
+ Void arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- public Set<FunctionSignature> getCalls() {
- return calls;
- }
+ @Override
+ public Void visitIndexDropStatement(IndexDropStatement del, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitCreateDataverseStatement(CreateDataverseStatement del, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitNodeGroupDropStatement(NodeGroupDropStatement del,
+ Void arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitIndexDropStatement(IndexDropStatement del, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitDataverseDropStatement(DataverseDropStatement del,
+ Void arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitNodeGroupDropStatement(NodeGroupDropStatement del, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitTypeDropStatement(TypeDropStatement del, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitDataverseDropStatement(DataverseDropStatement del, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitDisconnectFeedStatement(DisconnectFeedStatement del,
+ Void arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitTypeDropStatement(TypeDropStatement del, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visit(CreateFunctionStatement cfs, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitControlFeedStatement(ControlFeedStatement del, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitFunctionDropStatement(FunctionDropStatement del,
+ Void arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visit(CreateFunctionStatement cfs, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitCreateFeedStatement(CreateFeedStatement del, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitFunctionDropStatement(FunctionDropStatement del, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitConnectFeedStatement(ConnectFeedStatement del, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitBeginFeedStatement(BeginFeedStatement bf, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitDropFeedStatement(FeedDropStatement del, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- @Override
- public Void visitCompactStatement(CompactStatement del, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
- return null;
- }
+ @Override
+ public Void visitCompactStatement(CompactStatement del, Void arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
- }
+ }
}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/CloneAndSubstituteVariablesVisitor.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/CloneAndSubstituteVariablesVisitor.java
index d860b0c..1ec5f67 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/CloneAndSubstituteVariablesVisitor.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/CloneAndSubstituteVariablesVisitor.java
@@ -21,11 +21,12 @@
import edu.uci.ics.asterix.aql.base.Clause;
import edu.uci.ics.asterix.aql.base.Expression;
import edu.uci.ics.asterix.aql.base.IAqlExpression;
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
import edu.uci.ics.asterix.aql.expression.CallExpr;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.CompactStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
import edu.uci.ics.asterix.aql.expression.DatasetDecl;
@@ -35,6 +36,7 @@
import edu.uci.ics.asterix.aql.expression.DistinctClause;
import edu.uci.ics.asterix.aql.expression.DropStatement;
import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
+import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
import edu.uci.ics.asterix.aql.expression.FieldAccessor;
import edu.uci.ics.asterix.aql.expression.FieldBinding;
import edu.uci.ics.asterix.aql.expression.ForClause;
@@ -410,7 +412,7 @@
}
@Override
- public Pair<IAqlExpression, List<VariableSubstitution>> visitControlFeedStatement(ControlFeedStatement del,
+ public Pair<IAqlExpression, List<VariableSubstitution>> visitDisconnectFeedStatement(DisconnectFeedStatement del,
List<VariableSubstitution> arg) throws AsterixException {
// TODO Auto-generated method stub
return null;
@@ -578,7 +580,21 @@
}
@Override
- public Pair<IAqlExpression, List<VariableSubstitution>> visitBeginFeedStatement(BeginFeedStatement bf,
+ public Pair<IAqlExpression, List<VariableSubstitution>> visitCreateFeedStatement(CreateFeedStatement del,
+ List<VariableSubstitution> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Pair<IAqlExpression, List<VariableSubstitution>> visitConnectFeedStatement(ConnectFeedStatement del,
+ List<VariableSubstitution> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Pair<IAqlExpression, List<VariableSubstitution>> visitDropFeedStatement(FeedDropStatement del,
List<VariableSubstitution> arg) throws AsterixException {
// TODO Auto-generated method stub
return null;
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/InlineUdfsVisitor.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/InlineUdfsVisitor.java
index fc7acfe..1fed1ec 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/InlineUdfsVisitor.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/InlineUdfsVisitor.java
@@ -22,11 +22,12 @@
import edu.uci.ics.asterix.aql.base.Expression;
import edu.uci.ics.asterix.aql.base.Expression.Kind;
import edu.uci.ics.asterix.aql.base.IAqlExpression;
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
import edu.uci.ics.asterix.aql.expression.CallExpr;
+import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.DisconnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.CompactStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
import edu.uci.ics.asterix.aql.expression.DatasetDecl;
@@ -36,6 +37,7 @@
import edu.uci.ics.asterix.aql.expression.DistinctClause;
import edu.uci.ics.asterix.aql.expression.DropStatement;
import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
+import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
import edu.uci.ics.asterix.aql.expression.FieldAccessor;
import edu.uci.ics.asterix.aql.expression.FieldBinding;
import edu.uci.ics.asterix.aql.expression.ForClause;
@@ -500,7 +502,8 @@
}
@Override
- public Boolean visitControlFeedStatement(ControlFeedStatement del, List<FunctionDecl> arg) throws AsterixException {
+ public Boolean visitDisconnectFeedStatement(DisconnectFeedStatement del, List<FunctionDecl> arg)
+ throws AsterixException {
// TODO Auto-generated method stub
return null;
}
@@ -519,7 +522,19 @@
}
@Override
- public Boolean visitBeginFeedStatement(BeginFeedStatement bf, List<FunctionDecl> arg) throws AsterixException {
+ public Boolean visitCreateFeedStatement(CreateFeedStatement del, List<FunctionDecl> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Boolean visitConnectFeedStatement(ConnectFeedStatement del, List<FunctionDecl> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Boolean visitDropFeedStatement(FeedDropStatement del, List<FunctionDecl> arg) throws AsterixException {
// TODO Auto-generated method stub
return null;
}
diff --git a/asterix-aql/src/main/javacc/AQL.jj b/asterix-aql/src/main/javacc/AQL.jj
index 9080f9c..8b46dc2 100644
--- a/asterix-aql/src/main/javacc/AQL.jj
+++ b/asterix-aql/src/main/javacc/AQL.jj
@@ -86,6 +86,12 @@
}
};
+ private static class FunctionName {
+ public String dataverse = null;
+ public String library = null;
+ public String function = null;
+ }
+
private static String getHint(Token t) {
if (t.specialToken == null) {
return null;
@@ -234,6 +240,7 @@
| stmt = IndexSpecification()
| stmt = DataverseSpecification()
| stmt = FunctionSpecification()
+ | stmt = FeedSpecification()
)
{
return stmt;
@@ -247,7 +254,7 @@
TypeExpression typeExpr = null;
}
{
- "type" nameComponents = FunctionOrTypeName() ifNotExists = IfNotExists()
+ "type" nameComponents = TypeName() ifNotExists = IfNotExists()
"as" typeExpr = TypeExpr()
{
long numValues = -1;
@@ -325,32 +332,6 @@
ifNotExists);
}
- | "feed" <DATASET> nameComponents = QualifiedName()
- <LEFTPAREN> typeName = Identifier() <RIGHTPAREN>
- ifNotExists = IfNotExists()
- "using" adapterName = AdapterName() properties = Configuration()
- (appliedFunction = ApplyFunction())? primaryKeyFields = PrimaryKey()
- ( "on" nodeGroupName = Identifier() )?
- ( "hints" hints = Properties() )?
- ( "using" "compaction" "policy" compactionPolicy = CompactionPolicy() compactionPolicyProperties = Configuration() )?
- {
- FeedDetailsDecl fdd = new FeedDetailsDecl(adapterName,
- properties,
- appliedFunction,
- nodeGroupName != null
- ? new Identifier(nodeGroupName)
- : null,
- primaryKeyFields,
- compactionPolicy,
- compactionPolicyProperties);
- dsetDecl = new DatasetDecl(nameComponents.first,
- nameComponents.second,
- new Identifier(typeName),
- hints,
- DatasetType.FEED,
- fdd,
- ifNotExists);
- }
| ("internal")? <DATASET> nameComponents = QualifiedName()
<LEFTPAREN> typeName = Identifier() <RIGHTPAREN>
ifNotExists = IfNotExists()
@@ -474,33 +455,61 @@
boolean ifNotExists = false;
List<VarIdentifier> paramList = new ArrayList<VarIdentifier>();
String functionBody;
+ VarIdentifier var = null;
Expression functionBodyExpr;
Token beginPos;
Token endPos;
- Pair<Identifier,Identifier> nameComponents=null;
-
+ FunctionName fctName = null;
+
createNewScope();
}
{
- "function" nameComponents = FunctionOrTypeName()
+ "function" fctName = FunctionName()
ifNotExists = IfNotExists()
paramList = ParameterList()
<LEFTBRACE>
- {
- beginPos = token;
- }
+ {
+ beginPos = token;
+ }
functionBodyExpr = Expression() <RIGHTBRACE>
{
endPos = token;
functionBody = extractFragment(beginPos.beginLine, beginPos.beginColumn, endPos.beginLine, endPos.beginColumn);
- String dataverse = nameComponents.first.getValue();
- String functionName = nameComponents.second.getValue();
- signature = new FunctionSignature(dataverse, functionName, paramList.size());
+ // TODO use fctName.library
+ signature = new FunctionSignature(fctName.dataverse, fctName.function, paramList.size());
getCurrentScope().addFunctionDescriptor(signature, false);
return new CreateFunctionStatement(signature, paramList, functionBody, ifNotExists);
}
}
+CreateFeedStatement FeedSpecification() throws ParseException:
+{
+ Pair<Identifier,Identifier> nameComponents = null;
+ boolean ifNotExists = false;
+ String adaptorName = null;
+ Map<String,String> properties = null;
+ FunctionSignature appliedFunction = null;
+ CreateFeedStatement cfs = null;
+}
+{
+ (
+ "feed" nameComponents = QualifiedName()
+ ifNotExists = IfNotExists()
+ "using" adaptorName = AdapterName() properties = Configuration()
+ (appliedFunction = ApplyFunction())?
+ {
+ cfs = new CreateFeedStatement(nameComponents.first,
+ nameComponents.second, adaptorName, properties, appliedFunction, ifNotExists);
+ }
+
+ )
+ {
+ return cfs;
+ }
+}
+
+
+
List<VarIdentifier> ParameterList() throws ParseException:
{
List<VarIdentifier> paramList = new ArrayList<VarIdentifier>();
@@ -552,22 +561,34 @@
}
}
+String GetPolicy() throws ParseException:
+{
+ String policy = null;
+}
+{
+ "using" "policy" policy = Identifier()
+ {
+ return policy;
+ }
+
+}
+
FunctionSignature FunctionSignature() throws ParseException:
{
- Pair<Identifier,Identifier> pairId = null;
+ FunctionName fctName = null;
int arity = 0;
}
{
- pairId = FunctionOrTypeName() "@" <INTEGER_LITERAL>
+ fctName = FunctionName() "@" <INTEGER_LITERAL>
{
arity = new Integer(token.image);
if (arity < 0 && arity != FunctionIdentifier.VARARGS) {
throw new ParseException(" invalid arity:" + arity);
}
- String dataverse = pairId.first.getValue();
- String functionName = pairId.second.getValue();
- return new FunctionSignature(dataverse, functionName, arity);
+ // TODO use fctName.library
+ String fqFunctionName = fctName.library == null ? fctName.function : fctName.library + "#" + fctName.function;
+ return new FunctionSignature(fctName.dataverse, fqFunctionName, arity);
}
}
@@ -615,7 +636,7 @@
{
stmt = new NodeGroupDropStatement(new Identifier(id), ifExists);
}
- | "type" pairId = FunctionOrTypeName() ifExists = IfExists()
+ | "type" pairId = TypeName() ifExists = IfExists()
{
stmt = new TypeDropStatement(pairId.first, pairId.second, ifExists);
}
@@ -627,6 +648,10 @@
{
stmt = new FunctionDropStatement(funcSig, ifExists);
}
+ | "feed" pairId = QualifiedName() ifExists = IfExists()
+ {
+ stmt = new FeedDropStatement(pairId.first, pairId.second, ifExists);
+ }
)
{
return stmt;
@@ -809,31 +834,22 @@
Statement FeedStatement() throws ParseException:
{
- Pair<Identifier,Identifier> nameComponents = null;
+ Pair<Identifier,Identifier> feedNameComponents = null;
+ Pair<Identifier,Identifier> datasetNameComponents = null;
+
Map<String,String> configuration = null;
Statement stmt = null;
+ String policy = null;
}
{
(
- "begin" "feed" nameComponents = QualifiedName()
+ "connect" "feed" feedNameComponents = QualifiedName() "to" <DATASET> datasetNameComponents = QualifiedName() (policy = GetPolicy())?
{
- stmt = new BeginFeedStatement(nameComponents.first, nameComponents.second, getVarCounter());
+ stmt = new ConnectFeedStatement(feedNameComponents, datasetNameComponents, policy, getVarCounter());
}
- | "suspend" "feed" nameComponents = QualifiedName()
+ | "disconnect" "feed" feedNameComponents = QualifiedName() "from" <DATASET> datasetNameComponents = QualifiedName()
{
- stmt = new ControlFeedStatement(ControlFeedStatement.OperationType.SUSPEND, nameComponents.first, nameComponents.second);
- }
- | "resume" "feed" nameComponents = QualifiedName()
- {
- stmt = new ControlFeedStatement(ControlFeedStatement.OperationType.RESUME, nameComponents.first, nameComponents.second);
- }
- | "end" "feed" nameComponents = QualifiedName()
- {
- stmt = new ControlFeedStatement(ControlFeedStatement.OperationType.END, nameComponents.first, nameComponents.second);
- }
- | "alter" "feed" nameComponents = QualifiedName() "set" configuration = Configuration()
- {
- stmt = new ControlFeedStatement(ControlFeedStatement.OperationType.ALTER, nameComponents.first, nameComponents.second, configuration);
+ stmt = new DisconnectFeedStatement(feedNameComponents, datasetNameComponents);
}
)
{
@@ -1026,8 +1042,46 @@
}
}
+FunctionName FunctionName() throws ParseException:
+{
+ String first = null;
+ String second = null;
+ String third = null;
+ boolean secondAfterDot = false;
+}
+{
+ first = Identifier() ( "." second = Identifier()
+ {
+ secondAfterDot = true;
+ }
+ ("#" third = Identifier())? | "#" second = Identifier() )?
+ {
+ FunctionName result = new FunctionName();
+ if (second == null) {
+ result.dataverse = defaultDataverse;
+ result.library = null;
+ result.function = first;
+ } else if (third == null) {
+ if (secondAfterDot) {
+ result.dataverse = first;
+ result.library = null;
+ result.function = second;
+ } else {
+ result.dataverse = defaultDataverse;
+ result.library = first;
+ result.function = second;
+ }
+ } else {
+ result.dataverse = first;
+ result.library = second;
+ result.function = third;
+ }
+ return result;
+ }
+}
-Pair<Identifier,Identifier> FunctionOrTypeName() throws ParseException:
+
+Pair<Identifier,Identifier> TypeName() throws ParseException:
{
Pair<Identifier,Identifier> name = null;
}
@@ -1684,18 +1738,12 @@
List<Expression> argList = new ArrayList<Expression>();
Expression tmp;
int arity = 0;
- Pair<Identifier,Identifier> funcId = null;
- String funcName;
- String dataverse;
+ FunctionName funcName = null;
String hint = null;
- String id1 = null;
- String id2 = null;
}
{
- funcId = FunctionOrTypeName()
+ funcName = FunctionName()
{
- dataverse = funcId.first.getValue();
- funcName = funcId.second.getValue();
hint = getHint(token);
}
<LEFTPAREN> (tmp = Expression()
@@ -1710,9 +1758,12 @@
}
)*)? <RIGHTPAREN>
{
- FunctionSignature signature = lookupFunctionSignature(dataverse, funcName, arity);
+ // TODO use funcName.library
+ String fqFunctionName = funcName.library == null ? funcName.function : funcName.library + "#" + funcName.function;
+ FunctionSignature signature
+ = lookupFunctionSignature(funcName.dataverse, fqFunctionName, arity);
if (signature == null) {
- signature = new FunctionSignature(dataverse, funcName, arity);
+ signature = new FunctionSignature(funcName.dataverse, fqFunctionName, arity);
}
callExpr = new CallExpr(signature,argList);
if (hint != null && hint.startsWith(INDEXED_NESTED_LOOP_JOIN_HINT)) {
@@ -1722,6 +1773,7 @@
}
}
+
Expression DatasetAccessExpression() throws ParseException:
{
String funcName;
diff --git a/asterix-common/pom.xml b/asterix-common/pom.xml
index 7d1786e..8697ef3 100644
--- a/asterix-common/pom.xml
+++ b/asterix-common/pom.xml
@@ -1,18 +1,14 @@
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<!-- ! Copyright 2009-2013 by The Regents of the University of California
+ ! Licensed under the Apache License, Version 2.0 (the "License"); ! you may
+ not use this file except in compliance with the License. ! you may obtain
+ a copy of the License from ! ! http://www.apache.org/licenses/LICENSE-2.0
+ ! ! Unless required by applicable law or agreed to in writing, software !
+ distributed under the License is distributed on an "AS IS" BASIS, ! WITHOUT
+ WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. ! See the
+ License for the specific language governing permissions and ! limitations
+ under the License. ! -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<artifactId>asterix</artifactId>
@@ -70,6 +66,40 @@
<generateDirectory>${project.build.directory}/generated-sources/configuration</generateDirectory>
</configuration>
</execution>
+ <execution>
+ <id>cluster</id>
+ <goals>
+ <goal>generate</goal>
+ </goals>
+ <configuration>
+ <args>
+ <arg>-Xsetters</arg>
+ <arg>-Xvalue-constructor</arg>
+ </args>
+ <plugins>
+ <plugin>
+ <groupId>org.jvnet.jaxb2_commons</groupId>
+ <artifactId>jaxb2-basics</artifactId>
+ <version>0.6.2</version>
+ </plugin>
+ <plugin>
+ <groupId>org.jvnet.jaxb2_commons</groupId>
+ <artifactId>jaxb2-value-constructor</artifactId>
+ <version>3.0</version>
+ </plugin>
+ </plugins>
+ <schemaDirectory>src/main/resources/schema</schemaDirectory>
+ <schemaIncludes>
+ <include>cluster.xsd</include>
+ </schemaIncludes>
+ <generatePackage>edu.uci.ics.asterix.event.schema.cluster</generatePackage>
+ <generateDirectory>${project.build.directory}/generated-sources/cluster</generateDirectory>
+ <bindingDirectory>src/main/resources/schema</bindingDirectory>
+ <bindingIncludes>
+ <bindingInclude>jaxb-bindings.xjb</bindingInclude>
+ </bindingIncludes>
+ </configuration>
+ </execution>
</executions>
</plugin>
<plugin>
@@ -94,6 +124,16 @@
<dependencies>
<dependency>
+ <groupId>commons-io</groupId>
+ <artifactId>commons-io</artifactId>
+ <version>1.4</version>
+ </dependency>
+ <dependency>
+ <groupId>commons-httpclient</groupId>
+ <artifactId>commons-httpclient</artifactId>
+ <version>3.0.1</version>
+ </dependency>
+ <dependency>
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>algebricks-compiler</artifactId>
</dependency>
@@ -116,6 +156,14 @@
<artifactId>hyracks-storage-am-lsm-common</artifactId>
</dependency>
<dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-storage-am-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-api</artifactId>
+ </dependency>
+ <dependency>
<groupId>edu.uci.ics.asterix</groupId>
<artifactId>asterix-test-framework</artifactId>
<version>0.8.1-SNAPSHOT</version>
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java
index e975f60..63a8057 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java
@@ -27,6 +27,15 @@
private static final String EXTERNAL_APISERVER_KEY = "api.port";
private static int EXTERNAL_APISERVER_DEFAULT = 19002;
+ private static final String EXTERNAL_FEEDSERVER_KEY = "feed.port";
+ private static int EXTERNAL_FEEDSERVER_DEFAULT = 19003;
+
+ private static final String EXTERNAL_CC_JAVA_OPTS_KEY = "cc.java.opts";
+ private static String EXTERNAL_CC_JAVA_OPTS_DEFAULT = "-Xmx1024m";
+
+ private static final String EXTERNAL_NC_JAVA_OPTS_KEY = "nc.java.opts";
+ private static String EXTERNAL_NC_JAVA_OPTS_DEFAULT = "-Xmx1024m";
+
public AsterixExternalProperties(AsterixPropertiesAccessor accessor) {
super(accessor);
}
@@ -41,8 +50,23 @@
PropertyInterpreters.getIntegerPropertyInterpreter());
}
+ public int getFeedServerPort() {
+ return accessor.getProperty(EXTERNAL_FEEDSERVER_KEY, EXTERNAL_FEEDSERVER_DEFAULT,
+ PropertyInterpreters.getIntegerPropertyInterpreter());
+ }
+
public Level getLogLevel() {
return accessor.getProperty(EXTERNAL_LOGLEVEL_KEY, EXTERNAL_LOGLEVEL_DEFAULT,
PropertyInterpreters.getLevelPropertyInterpreter());
}
+
+ public String getNCJavaParams() {
+ return accessor.getProperty(EXTERNAL_NC_JAVA_OPTS_KEY, EXTERNAL_NC_JAVA_OPTS_DEFAULT,
+ PropertyInterpreters.getStringPropertyInterpreter());
+ }
+
+ public String getCCJavaParams() {
+ return accessor.getProperty(EXTERNAL_CC_JAVA_OPTS_KEY, EXTERNAL_CC_JAVA_OPTS_DEFAULT,
+ PropertyInterpreters.getStringPropertyInterpreter());
+ }
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixMetadataProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixMetadataProperties.java
index 323f5f8..1d68dfc 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixMetadataProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixMetadataProperties.java
@@ -23,6 +23,10 @@
super(accessor);
}
+ public String getInstanceName() {
+ return accessor.getInstanceName();
+ }
+
public String getMetadataNodeName() {
return accessor.getMetadataNodeName();
}
@@ -38,9 +42,12 @@
public Set<String> getNodeNames() {
return accessor.getNodeNames();
}
-
- public String getCoredumpPath(String nodeId){
+
+ public String getCoredumpPath(String nodeId) {
return accessor.getCoredumpPath(nodeId);
}
+ public Map<String, String> getCoredumpPaths() {
+ return accessor.getCoredumpConfig();
+ }
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixPropertiesAccessor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixPropertiesAccessor.java
index 2bd292e..7654aa3 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixPropertiesAccessor.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixPropertiesAccessor.java
@@ -39,6 +39,7 @@
public class AsterixPropertiesAccessor {
private static final Logger LOGGER = Logger.getLogger(AsterixPropertiesAccessor.class.getName());
+ private final String instanceName;
private final String metadataNodeName;
private final Set<String> nodeNames;
private final Map<String, String[]> stores;
@@ -69,6 +70,7 @@
} catch (JAXBException e) {
throw new AsterixException("Failed to read configuration file " + fileName);
}
+ instanceName = asterixConfiguration.getInstanceName();
metadataNodeName = asterixConfiguration.getMetadataNode();
stores = new HashMap<String, String[]>();
List<Store> configuredStores = asterixConfiguration.getStore();
@@ -113,8 +115,26 @@
return coredumpConfig.get(nodeId);
}
- public String getTransactionLogDir(String nodeId) {
- return transactionLogDirs.get(nodeId);
+ public Map<String, String> getTransactionLogDirs() {
+ return transactionLogDirs;
+ }
+
+ public Map<String, String> getCoredumpConfig() {
+ return coredumpConfig;
+ }
+
+ public void putCoredumpPaths(String nodeId, String coredumpPath) {
+ if (coredumpConfig.containsKey(nodeId)) {
+ throw new IllegalStateException("Cannot override value for coredump path");
+ }
+ coredumpConfig.put(nodeId, coredumpPath);
+ }
+
+ public void putTransactionLogDir(String nodeId, String txnLogDir) {
+ if (transactionLogDirs.containsKey(nodeId)) {
+ throw new IllegalStateException("Cannot override value for txnLogDir");
+ }
+ transactionLogDirs.put(nodeId, txnLogDir);
}
public <T> T getProperty(String property, T defaultValue, IPropertyInterpreter<T> interpreter) {
@@ -137,4 +157,9 @@
+ "'.\n See the description: \n" + p.getDescription() + "\nDefault = " + defaultValue);
}
}
+
+ public String getInstanceName() {
+ return instanceName;
+ }
+
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
index a1dd52a..978577a 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
@@ -14,6 +14,8 @@
*/
package edu.uci.ics.asterix.common.config;
+import java.util.Map;
+
public class AsterixTransactionProperties extends AbstractAsterixProperties {
private static final String TXN_LOG_BUFFER_NUMPAGES_KEY = "txn.log.buffer.numpages";
@@ -51,7 +53,11 @@
}
public String getLogDirectory(String nodeId) {
- return accessor.getTransactionLogDir(nodeId);
+ return accessor.getTransactionLogDirs().get(nodeId);
+ }
+
+ public Map<String, String> getLogDirectories() {
+ return accessor.getTransactionLogDirs();
}
public int getLogBufferNumPages() {
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java
index 60cf6e5..8481817 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java
@@ -17,28 +17,20 @@
public class DatasetConfig {
/*
- * We have three kinds of datasets. INTERNAL: A dataset with data persisted
+ * We have two kinds of datasets. INTERNAL: A dataset with data persisted
* in ASTERIX storage. The dataset is populated either using a load
* statement or using insert statement. EXTERNAL: A dataset with data
* residing outside ASTERIX. As such ASTERIX does not maintain any indexes
* on the data. The data for the dataset is fetched as and when required
- * from an external data source using an adapter. FEED : A dataset that can
- * be considered as a hybrid of INTERNAL and EXTERNAL dataset. A FEED
- * dataset is INTERNAL in the sense that the data is persisted within
- * ASTERIX storage and has associated indexes that are maintained by
- * ASTERIX. However the dataset is initially populated with data fetched
- * from an external datasource using an adapter, in a manner similar to an
- * EXTERNAL dataset. A FEED dataset continuously receives data from the
- * associated adapter.
+ * from an external data source using an adapter.
*/
public enum DatasetType {
INTERNAL,
- EXTERNAL,
- FEED
+ EXTERNAL
}
public enum IndexType {
- BTREE,
+ BTREE,
RTREE,
SINGLE_PARTITION_WORD_INVIX,
SINGLE_PARTITION_NGRAM_INVIX,
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
index bc19543..26a79c3 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
@@ -75,6 +75,7 @@
}
}
} catch (Exception e) {
+ e.printStackTrace();
throw new HyracksDataException(e);
}
System.arraycopy(buffer.array(), 0, writeBuffer.array(), 0, buffer.capacity());
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMTreeInsertDeleteOperatorDescriptor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMTreeInsertDeleteOperatorDescriptor.java
index e58bbfa..6f6a7b2 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMTreeInsertDeleteOperatorDescriptor.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMTreeInsertDeleteOperatorDescriptor.java
@@ -56,4 +56,19 @@
recordDescProvider, op, isPrimary);
}
+ public boolean isPrimary() {
+ return isPrimary;
+ }
+
+ public int[] getFieldPermutations() {
+ return fieldPermutation;
+ }
+
+ public IndexOperation getIndexOperation() {
+ return op;
+ }
+
+ public IBinaryComparatorFactory[] getComparatorFactories() {
+ return comparatorFactories;
+ }
}
diff --git a/asterix-common/src/main/resources/schema/asterix-conf.xsd b/asterix-common/src/main/resources/schema/asterix-conf.xsd
index f461723..3033893 100644
--- a/asterix-common/src/main/resources/schema/asterix-conf.xsd
+++ b/asterix-common/src/main/resources/schema/asterix-conf.xsd
@@ -5,7 +5,8 @@
<!-- definition of simple types -->
-
+
+ <xs:element name="instanceName" type="xs:string" />
<xs:element name="metadataNode" type="xs:string" />
<xs:element name="coredumpPath" type="xs:string" />
<xs:element name="storeDirs" type="xs:string" />
@@ -13,9 +14,9 @@
<xs:element name="name" type="xs:string" />
<xs:element name="value" type="xs:string" />
<xs:element name="description" type="xs:string" />
- <xs:element name="txnLogDirPath" type="xs:string" />
-
-
+ <xs:element name="txnLogDirPath" type="xs:string" />
+
+
<!-- definition of complex elements -->
<xs:element name="store">
<xs:complexType>
@@ -34,7 +35,7 @@
</xs:sequence>
</xs:complexType>
</xs:element>
-
+
<xs:element name="transactionLogDir">
<xs:complexType>
<xs:sequence>
@@ -58,10 +59,11 @@
<xs:element name="asterixConfiguration">
<xs:complexType>
<xs:sequence>
- <xs:element ref="mg:metadataNode" minOccurs="0"/>
+ <xs:element ref="mg:instanceName" minOccurs="0" />
+ <xs:element ref="mg:metadataNode" minOccurs="0" />
<xs:element ref="mg:store" maxOccurs="unbounded" />
<xs:element ref="mg:coredump" maxOccurs="unbounded" />
- <xs:element ref="mg:transactionLogDir" maxOccurs="unbounded"/>
+ <xs:element ref="mg:transactionLogDir" maxOccurs="unbounded" />
<xs:element ref="mg:property" minOccurs="0" maxOccurs="unbounded" />
</xs:sequence>
</xs:complexType>
diff --git a/asterix-events/src/main/resources/schema/cluster.xsd b/asterix-common/src/main/resources/schema/cluster.xsd
similarity index 79%
rename from asterix-events/src/main/resources/schema/cluster.xsd
rename to asterix-common/src/main/resources/schema/cluster.xsd
index f0d5bd9..8c317d8 100644
--- a/asterix-events/src/main/resources/schema/cluster.xsd
+++ b/asterix-common/src/main/resources/schema/cluster.xsd
@@ -3,7 +3,8 @@
xmlns:cl="cluster" targetNamespace="cluster" elementFormDefault="qualified">
<!-- definition of simple types -->
- <xs:element name="name" type="xs:string" />
+ <xs:element name="instance_name" type="xs:string" />
+ <xs:element name="cluster_name" type="xs:string" />
<xs:element name="log_dir" type="xs:string" />
<xs:element name="txn_log_dir" type="xs:string" />
<xs:element name="id" type="xs:string" />
@@ -21,6 +22,9 @@
<xs:element name="client_port" type="xs:integer" />
<xs:element name="cluster_port" type="xs:integer" />
<xs:element name="http_port" type="xs:integer" />
+ <xs:element name="debug_port" type="xs:integer" />
+ <xs:element name="metadata_node" type="xs:string" />
+
<!-- definition of complex elements -->
<xs:element name="working_dir">
@@ -40,9 +44,10 @@
<xs:element ref="cl:cluster_ip" />
<xs:element ref="cl:java_home" minOccurs="0" />
<xs:element ref="cl:log_dir" minOccurs="0" />
- <xs:element ref="cl:client_port" />
- <xs:element ref="cl:cluster_port" />
- <xs:element ref="cl:http_port" />
+ <xs:element ref="cl:client_port" />
+ <xs:element ref="cl:cluster_port" />
+ <xs:element ref="cl:http_port" />
+ <xs:element ref="cl:debug_port" minOccurs="0" />
</xs:sequence>
</xs:complexType>
</xs:element>
@@ -74,6 +79,15 @@
<xs:element ref="cl:txn_log_dir" minOccurs="0" />
<xs:element ref="cl:store" minOccurs="0" />
<xs:element ref="cl:iodevices" minOccurs="0" />
+ <xs:element ref="cl:debug_port" minOccurs="0" />
+ </xs:sequence>
+ </xs:complexType>
+ </xs:element>
+
+ <xs:element name="substitute_nodes">
+ <xs:complexType>
+ <xs:sequence>
+ <xs:element ref="cl:node" maxOccurs="unbounded" />
</xs:sequence>
</xs:complexType>
</xs:element>
@@ -81,7 +95,8 @@
<xs:element name="cluster">
<xs:complexType>
<xs:sequence>
- <xs:element ref="cl:name" />
+ <xs:element ref="cl:instance_name" />
+ <xs:element ref="cl:cluster_name" />
<xs:element ref="cl:username" />
<xs:element ref="cl:env" minOccurs="0" />
<xs:element ref="cl:java_home" minOccurs="0" />
@@ -90,8 +105,10 @@
<xs:element ref="cl:store" minOccurs="0" />
<xs:element ref="cl:iodevices" minOccurs="0" />
<xs:element ref="cl:working_dir" />
+ <xs:element ref="cl:metadata_node" />
<xs:element ref="cl:master_node" />
<xs:element ref="cl:node" maxOccurs="unbounded" />
+ <xs:element ref="cl:substitute_nodes" />
</xs:sequence>
</xs:complexType>
</xs:element>
diff --git a/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestHelper.java b/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestHelper.java
new file mode 100644
index 0000000..5777d0a
--- /dev/null
+++ b/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestHelper.java
@@ -0,0 +1,16 @@
+package edu.uci.ics.asterix.test.aql;
+
+import java.util.List;
+
+public final class TestHelper {
+
+ public static boolean isInPrefixList(List<String> prefixList, String s) {
+ for (String s2 : prefixList) {
+ if (s.startsWith(s2)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+}
diff --git a/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java b/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
index 861cce5..d41815d 100644
--- a/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
+++ b/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
@@ -366,11 +366,15 @@
case "mgx":
executeManagixCommand(statement);
break;
+ case "sleep":
+ Thread.sleep(Long.parseLong(statement.trim()));
+ break;
default:
throw new IllegalArgumentException("No statements of type " + ctx.getType());
}
} catch (Exception e) {
+ e.printStackTrace();
if (cUnit.getExpectedError().isEmpty()) {
throw new Exception("Test \"" + testFile + "\" FAILED!", e);
}
diff --git a/asterix-events/pom.xml b/asterix-events/pom.xml
index f221df2..8457198 100644
--- a/asterix-events/pom.xml
+++ b/asterix-events/pom.xml
@@ -1,196 +1,207 @@
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
- <parent>
- <artifactId>asterix</artifactId>
- <groupId>edu.uci.ics.asterix</groupId>
- <version>0.8.1-SNAPSHOT</version>
- </parent>
- <groupId>edu.uci.ics.asterix</groupId>
- <artifactId>asterix-events</artifactId>
- <version>0.8.1-SNAPSHOT</version>
+<!-- ! Copyright 2009-2013 by The Regents of the University of California
+ ! Licensed under the Apache License, Version 2.0 (the "License"); ! you may
+ not use this file except in compliance with the License. ! you may obtain
+ a copy of the License from ! ! http://www.apache.org/licenses/LICENSE-2.0
+ ! ! Unless required by applicable law or agreed to in writing, software !
+ distributed under the License is distributed on an "AS IS" BASIS, ! WITHOUT
+ WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. ! See the
+ License for the specific language governing permissions and ! limitations
+ under the License. ! -->
- <properties>
- <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
- </properties>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <artifactId>asterix</artifactId>
+ <groupId>edu.uci.ics.asterix</groupId>
+ <version>0.8.1-SNAPSHOT</version>
+ </parent>
+ <groupId>edu.uci.ics.asterix</groupId>
+ <artifactId>asterix-events</artifactId>
+ <version>0.8.1-SNAPSHOT</version>
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-compiler-plugin</artifactId>
- <version>2.0.2</version>
- <configuration>
- <source>1.6</source>
- <target>1.6</target>
- </configuration>
- </plugin>
- <plugin>
- <groupId>org.jvnet.jaxb2.maven2</groupId>
- <artifactId>maven-jaxb2-plugin</artifactId>
- <executions>
- <execution>
- <id>event</id>
- <goals>
- <goal>generate</goal>
- </goals>
- <configuration>
- <args>
- <arg>-Xsetters</arg>
- <arg>-Xvalue-constructor</arg>
- </args>
- <plugins>
- <plugin>
- <groupId>org.jvnet.jaxb2_commons</groupId>
- <artifactId>jaxb2-basics</artifactId>
- <version>0.6.2</version>
- </plugin>
- <plugin>
- <groupId>org.jvnet.jaxb2_commons</groupId>
- <artifactId>jaxb2-value-constructor</artifactId>
- <version>3.0</version>
- </plugin>
- </plugins>
- <schemaDirectory>src/main/resources/schema</schemaDirectory>
- <schemaIncludes>
- <include>event.xsd</include>
- </schemaIncludes>
- <generatePackage>edu.uci.ics.asterix.event.schema.event</generatePackage>
- <generateDirectory>${project.build.directory}/generated-sources/event</generateDirectory>
- </configuration>
- </execution>
- <execution>
- <id>pattern</id>
- <goals>
- <goal>generate</goal>
- </goals>
- <configuration>
- <args>
- <arg>-Xsetters</arg>
- <arg>-Xvalue-constructor</arg>
- </args>
- <plugins>
- <plugin>
- <groupId>org.jvnet.jaxb2_commons</groupId>
- <artifactId>jaxb2-basics</artifactId>
- <version>0.6.2</version>
- </plugin>
- <plugin>
- <groupId>org.jvnet.jaxb2_commons</groupId>
- <artifactId>jaxb2-value-constructor</artifactId>
- <version>3.0</version>
- </plugin>
- </plugins>
- <schemaDirectory>src/main/resources/schema</schemaDirectory>
- <schemaIncludes>
- <include>pattern.xsd</include>
- </schemaIncludes>
- <generatePackage>edu.uci.ics.asterix.event.schema.pattern</generatePackage>
- <generateDirectory>${project.build.directory}/generated-sources/pattern</generateDirectory>
- </configuration>
- </execution>
- <execution>
- <id>cluster</id>
- <goals>
- <goal>generate</goal>
- </goals>
- <configuration>
- <args>
- <arg>-Xsetters</arg>
- <arg>-Xvalue-constructor</arg>
- </args>
- <plugins>
- <plugin>
- <groupId>org.jvnet.jaxb2_commons</groupId>
- <artifactId>jaxb2-basics</artifactId>
- <version>0.6.2</version>
- </plugin>
- <plugin>
- <groupId>org.jvnet.jaxb2_commons</groupId>
- <artifactId>jaxb2-value-constructor</artifactId>
- <version>3.0</version>
- </plugin>
- </plugins>
- <schemaDirectory>src/main/resources/schema</schemaDirectory>
- <schemaIncludes>
- <include>cluster.xsd</include>
- </schemaIncludes>
- <generatePackage>edu.uci.ics.asterix.event.schema.cluster</generatePackage>
- <generateDirectory>${project.build.directory}/generated-sources/cluster</generateDirectory>
- <bindingDirectory>src/main/resources/schema</bindingDirectory>
- <bindingIncludes>
- <bindingInclude>jaxb-bindings.xjb</bindingInclude>
- </bindingIncludes>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <artifactId>maven-assembly-plugin</artifactId>
- <version>2.2-beta-2</version>
- <executions>
- <execution>
- <configuration>
- <descriptor>src/main/assembly/binary-assembly.xml</descriptor>
- </configuration>
- <phase>package</phase>
- <goals>
- <goal>attached</goal>
- </goals>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
+ <properties>
+ <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+ </properties>
- <dependencies>
- <dependency>
- <groupId>junit</groupId>
- <artifactId>junit</artifactId>
- <version>4.8.1</version>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>args4j</groupId>
- <artifactId>args4j</artifactId>
- <version>2.0.12</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.jvnet.jaxb2_commons</groupId>
- <artifactId>jaxb2-value-constructor</artifactId>
- <version>3.0</version>
- </dependency>
- <dependency>
- <groupId>commons-io</groupId>
- <artifactId>commons-io</artifactId>
- <version>1.4</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.asterix</groupId>
- <artifactId>asterix-common</artifactId>
- <version>0.8.1-SNAPSHOT</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
- <version>1.2.9</version>
- </dependency>
- </dependencies>
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>2.0.2</version>
+ <configuration>
+ <source>1.6</source>
+ <target>1.6</target>
+ </configuration>
+ </plugin>
+ <plugin>
+ <groupId>org.jvnet.jaxb2.maven2</groupId>
+ <artifactId>maven-jaxb2-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>event</id>
+ <goals>
+ <goal>generate</goal>
+ </goals>
+ <configuration>
+ <args>
+ <arg>-Xsetters</arg>
+ <arg>-Xvalue-constructor</arg>
+ </args>
+ <plugins>
+ <plugin>
+ <groupId>org.jvnet.jaxb2_commons</groupId>
+ <artifactId>jaxb2-basics</artifactId>
+ <version>0.6.2</version>
+ </plugin>
+ <plugin>
+ <groupId>org.jvnet.jaxb2_commons</groupId>
+ <artifactId>jaxb2-value-constructor</artifactId>
+ <version>3.0</version>
+ </plugin>
+ </plugins>
+ <schemaDirectory>src/main/resources/schema</schemaDirectory>
+ <schemaIncludes>
+ <include>event.xsd</include>
+ </schemaIncludes>
+ <generatePackage>edu.uci.ics.asterix.event.schema.event</generatePackage>
+ <generateDirectory>${project.build.directory}/generated-sources/event</generateDirectory>
+ </configuration>
+ </execution>
+ <execution>
+ <id>configuration</id>
+ <goals>
+ <goal>generate</goal>
+ </goals>
+ <configuration>
+ <args>
+ <arg>-Xsetters</arg>
+ <arg>-Xvalue-constructor</arg>
+ </args>
+ <plugins>
+ <plugin>
+ <groupId>org.jvnet.jaxb2_commons</groupId>
+ <artifactId>jaxb2-basics</artifactId>
+ <version>0.6.2</version>
+ </plugin>
+ <plugin>
+ <groupId>org.jvnet.jaxb2_commons</groupId>
+ <artifactId>jaxb2-value-constructor</artifactId>
+ <version>3.0</version>
+ </plugin>
+ </plugins>
+ <schemaDirectory>src/main/resources/schema</schemaDirectory>
+ <schemaIncludes>
+ <include>installer-conf.xsd</include>
+ </schemaIncludes>
+ <generatePackage>edu.uci.ics.asterix.installer.schema.conf</generatePackage>
+ <generateDirectory>${project.build.directory}/generated-sources/configuration</generateDirectory>
+ </configuration>
+ </execution>
+ <execution>
+ <id>pattern</id>
+ <goals>
+ <goal>generate</goal>
+ </goals>
+ <configuration>
+ <args>
+ <arg>-Xsetters</arg>
+ <arg>-Xvalue-constructor</arg>
+ </args>
+ <plugins>
+ <plugin>
+ <groupId>org.jvnet.jaxb2_commons</groupId>
+ <artifactId>jaxb2-basics</artifactId>
+ <version>0.6.2</version>
+ </plugin>
+ <plugin>
+ <groupId>org.jvnet.jaxb2_commons</groupId>
+ <artifactId>jaxb2-value-constructor</artifactId>
+ <version>3.0</version>
+ </plugin>
+ </plugins>
+ <schemaDirectory>src/main/resources/schema</schemaDirectory>
+ <schemaIncludes>
+ <include>pattern.xsd</include>
+ </schemaIncludes>
+ <generatePackage>edu.uci.ics.asterix.event.schema.pattern</generatePackage>
+ <generateDirectory>${project.build.directory}/generated-sources/pattern</generateDirectory>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <artifactId>maven-assembly-plugin</artifactId>
+ <version>2.2-beta-2</version>
+ <executions>
+ <execution>
+ <configuration>
+ <descriptor>src/main/assembly/binary-assembly.xml</descriptor>
+ </configuration>
+ <phase>package</phase>
+ <goals>
+ <goal>attached</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+ <dependencies>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <version>4.8.1</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>args4j</groupId>
+ <artifactId>args4j</artifactId>
+ <version>2.0.12</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.jvnet.jaxb2_commons</groupId>
+ <artifactId>jaxb2-value-constructor</artifactId>
+ <version>3.0</version>
+ </dependency>
+ <dependency>
+ <groupId>commons-io</groupId>
+ <artifactId>commons-io</artifactId>
+ <version>1.4</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.zookeeper</groupId>
+ <artifactId>zookeeper</artifactId>
+ <version>3.4.5</version>
+ <exclusions>
+ <exclusion>
+ <groupId>com.sun.jmx</groupId>
+ <artifactId>jmxri</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>com.sun.jdmk</groupId>
+ <artifactId>jmxtools</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>javax.jms</groupId>
+ <artifactId>jms</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.asterix</groupId>
+ <artifactId>asterix-common</artifactId>
+ <version>0.8.1-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>log4j</groupId>
+ <artifactId>log4j</artifactId>
+ <version>1.2.9</version>
+ </dependency>
+ </dependencies>
</project>
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/driver/EventDriver.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/driver/EventDriver.java
index 641b37b..2317cfa 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/driver/EventDriver.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/driver/EventDriver.java
@@ -29,7 +29,7 @@
import edu.uci.ics.asterix.event.management.DefaultOutputHandler;
import edu.uci.ics.asterix.event.management.EventUtil;
-import edu.uci.ics.asterix.event.management.EventrixClient;
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
import edu.uci.ics.asterix.event.management.IOutputHandler;
import edu.uci.ics.asterix.event.management.Randomizer;
import edu.uci.ics.asterix.event.schema.cluster.Cluster;
@@ -41,7 +41,7 @@
public class EventDriver {
public static final String CLIENT_NODE_ID = "client_node";
- public static final Node CLIENT_NODE = new Node(CLIENT_NODE_ID, "127.0.0.1", null, null, null, null, null);
+ public static final Node CLIENT_NODE = new Node(CLIENT_NODE_ID, "127.0.0.1", null, null, null, null, null, null);
private static String eventsDir;
private static Events events;
@@ -87,15 +87,6 @@
}
- public static EventrixClient getClient(String eventsDir, Cluster cluster, boolean dryRun) throws Exception {
- return new EventrixClient(eventsDir, cluster, dryRun, new DefaultOutputHandler());
- }
-
- public static EventrixClient getClient(String eventsDir, Cluster cluster, boolean dryRun,
- IOutputHandler outputHandler) throws Exception {
- return new EventrixClient(eventsDir, cluster, dryRun, outputHandler);
- }
-
public static void main(String[] args) throws Exception {
String eventsHome = System.getenv("EVENT_HOME");
if (eventsHome == null) {
@@ -119,9 +110,9 @@
if (!eventConfig.dryRun) {
prepare(cluster);
}
- EventrixClient client = new EventrixClient(eventsDir, cluster, eventConfig.dryRun,
- new DefaultOutputHandler());
- client.submit(patterns);
+ //AsterixEventServiceClient client = new AsterixEventServiceClient(eventsDir, cluster, eventConfig.dryRun,
+ // new DefaultOutputHandler());
+ // client.submit(patterns);
if (!eventConfig.dryRun) {
cleanup(cluster);
}
@@ -156,4 +147,5 @@
}
EventUtil.executeLocalScript(clientNode, eventsDir + "/" + "events" + "/" + "cleanup.sh", args);
}
+
}
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/xml/PatternParser.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/error/EventException.java
similarity index 74%
rename from asterix-events/src/main/java/edu/uci/ics/asterix/event/xml/PatternParser.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/error/EventException.java
index af46e63..83ed5f4 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/xml/PatternParser.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/error/EventException.java
@@ -12,12 +12,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.event.xml;
+package edu.uci.ics.asterix.event.error;
-public class PatternParser {
+public class EventException extends Exception {
- public static void parsePattern(String path){
-
- }
+ private static final long serialVersionUID = 1L;
+
+ public EventException(String message) {
+ super(message);
+ }
+
}
-
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/error/OutputHandler.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/error/OutputHandler.java
new file mode 100644
index 0000000..3bc795a
--- /dev/null
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/error/OutputHandler.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.event.error;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.Properties;
+
+import edu.uci.ics.asterix.event.management.IOutputHandler;
+import edu.uci.ics.asterix.event.management.OutputAnalysis;
+import edu.uci.ics.asterix.event.model.EventList.EventType;
+import edu.uci.ics.asterix.event.schema.pattern.Event;
+
+public class OutputHandler implements IOutputHandler {
+
+ public static IOutputHandler INSTANCE = new OutputHandler();
+
+ private OutputHandler() {
+
+ }
+
+ public OutputAnalysis reportEventOutput(Event event, String output) {
+
+ EventType eventType = EventType.valueOf(event.getType().toUpperCase());
+ boolean ignore = true;
+ String trimmedOutput = output.trim();
+ StringBuffer errorMessage = new StringBuffer();
+ switch (eventType) {
+ case FILE_TRANSFER:
+ if (trimmedOutput.length() > 0) {
+ if (output.contains("Permission denied") || output.contains("cannot find or open")) {
+ ignore = false;
+ break;
+ }
+ }
+ break;
+
+ case BACKUP:
+ case RESTORE:
+ if (trimmedOutput.length() > 0) {
+ if (trimmedOutput.contains("AccessControlException")) {
+ errorMessage.append("Insufficient permissions on back up directory");
+ ignore = false;
+ }
+ if (output.contains("does not exist") || output.contains("File exist")
+ || (output.contains("No such file or directory"))) {
+ ignore = true;
+ } else {
+ ignore = false;
+ }
+ }
+ break;
+
+ case NODE_INFO:
+ Properties p = new Properties();
+ try {
+ p.load(new ByteArrayInputStream(trimmedOutput.getBytes()));
+ } catch (IOException e) {
+ }
+ String javaVersion = (String) p.get("java_version");
+ if (p.get("java_version") == null) {
+ errorMessage.append("Java not installed on " + event.getNodeid().getValue().getAbsvalue());
+ ignore = false;
+ } else if (!javaVersion.contains("1.7")) {
+ errorMessage.append("Asterix requires Java 1.7.x. Incompatible version found on "
+ + event.getNodeid().getValue().getAbsvalue() + "\n");
+ ignore = false;
+ }
+ break;
+ }
+ if (ignore) {
+ return new OutputAnalysis(true, null);
+ } else {
+ return new OutputAnalysis(false, errorMessage.toString());
+ }
+ }
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/VerificationUtil.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/error/VerificationUtil.java
similarity index 78%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/VerificationUtil.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/error/VerificationUtil.java
index 5be3f89..b329171 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/VerificationUtil.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/error/VerificationUtil.java
@@ -1,5 +1,5 @@
/*
- * Copyright 2009-2013 by The Regents of the University of California
+ * Copyright 2009-2012 by The Regents of the University of California
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* you may obtain a copy of the License from
@@ -12,30 +12,29 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.installer.error;
+package edu.uci.ics.asterix.event.error;
import java.io.File;
import java.util.ArrayList;
import java.util.List;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.model.AsterixRuntimeState;
+import edu.uci.ics.asterix.event.model.ProcessInfo;
import edu.uci.ics.asterix.event.schema.cluster.Cluster;
import edu.uci.ics.asterix.event.schema.cluster.Node;
-import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.model.AsterixRuntimeState;
-import edu.uci.ics.asterix.installer.model.ProcessInfo;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
public class VerificationUtil {
- private static final String VERIFY_SCRIPT_PATH = InstallerDriver
- .getManagixHome()
- + File.separator
- + InstallerDriver.MANAGIX_INTERNAL_DIR
+ private static final String VERIFY_SCRIPT_PATH = AsterixEventService
+ .getEventHome()
+ File.separator
+ "scripts"
- + File.separator + "verify.sh";
+ + File.separator
+ + "verify.sh";
public static AsterixRuntimeState getAsterixRuntimeState(
AsterixInstance instance) throws Exception {
@@ -49,8 +48,8 @@
args.add(instance.getName() + "_" + node.getId());
}
Thread.sleep(2000);
- String output = InstallerUtil.executeLocalScript(VERIFY_SCRIPT_PATH,
- args);
+ String output = AsterixEventServiceUtil.executeLocalScript(
+ VERIFY_SCRIPT_PATH, args);
boolean ccRunning = true;
List<String> failedNCs = new ArrayList<String>();
String[] infoFields;
@@ -97,10 +96,10 @@
for (String failedNC : state.getFailedNCs()) {
summary.append(failedNC + "\n");
}
- instance.setState(State.UNUSABLE);
+ // instance.setState(State.UNUSABLE);
}
if (!(instance.getState().equals(State.UNUSABLE))) {
- instance.setState(State.ACTIVE);
+ instance.setState(State.ACTIVE);
}
} else {
if (state.getProcesses() != null && state.getProcesses().size() > 0) {
@@ -108,9 +107,9 @@
for (ProcessInfo pInfo : state.getProcesses()) {
summary.append(pInfo + "\n");
}
- instance.setState(State.UNUSABLE);
+ // instance.setState(State.UNUSABLE);
} else {
- instance.setState(State.INACTIVE);
+ // instance.setState(State.INACTIVE);
}
}
state.setSummary(summary.toString());
@@ -130,9 +129,7 @@
errorCheck.append("\n HDFS backup directory not configured");
}
if (errorCheck.length() > 0) {
- throw new Exception("Incomplete hdfs configuration in "
- + InstallerDriver.getManagixHome() + File.separator
- + InstallerDriver.MANAGIX_CONF_XML + errorCheck);
+ throw new Exception("Incomplete hdfs configuration" + errorCheck);
}
}
}
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventrixClient.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/AsterixEventServiceClient.java
similarity index 64%
rename from asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventrixClient.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/management/AsterixEventServiceClient.java
index 03f4061..8947247 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventrixClient.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/AsterixEventServiceClient.java
@@ -16,6 +16,7 @@
import java.io.File;
import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.LinkedBlockingQueue;
@@ -24,6 +25,7 @@
import javax.xml.bind.JAXBContext;
import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
import javax.xml.bind.Unmarshaller;
import edu.uci.ics.asterix.event.driver.EventDriver;
@@ -35,10 +37,12 @@
import edu.uci.ics.asterix.event.schema.pattern.Pattern;
import edu.uci.ics.asterix.event.schema.pattern.Patterns;
import edu.uci.ics.asterix.event.schema.pattern.Value;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.installer.schema.conf.Configuration;
-public class EventrixClient {
+public class AsterixEventServiceClient {
- private static final Logger LOGGER = Logger.getLogger(EventrixClient.class.getName());
+ private static final Logger LOGGER = Logger.getLogger(AsterixEventServiceClient.class.getName());
private EventTask[] tasks;
private boolean dryRun = false;
@@ -48,21 +52,26 @@
private IPatternListener listener;
private IOutputHandler outputHandler;
private Events events;
- private String eventsDir;
+ private String eventsHomeDir;
+ private Configuration configuration;
- public EventrixClient(String eventsDir, Cluster cluster, boolean dryRun, IOutputHandler outputHandler)
- throws Exception {
- this.eventsDir = eventsDir;
+ public AsterixEventServiceClient(Configuration configuration, String eventsHomeDir, Cluster cluster,
+ boolean transferArtifacts, boolean dryRun, IOutputHandler outputHandler) throws Exception {
+ this.eventsHomeDir = eventsHomeDir;
this.events = initializeEvents();
this.cluster = cluster;
this.dryRun = dryRun;
+ this.configuration = configuration;
this.outputHandler = outputHandler;
- if (!dryRun) {
- initializeCluster(eventsDir);
+ if (!dryRun && transferArtifacts) {
+ initializeCluster(getEventsDir());
}
}
public void submit(Patterns patterns) throws Exception {
+ if (patterns.getPattern().isEmpty()) {
+ return;
+ }
initTasks(patterns);
try {
waitForCompletion();
@@ -103,7 +112,7 @@
}
public String getEventsDir() {
- return eventsDir;
+ return eventsHomeDir + File.separator + AsterixEventServiceUtil.EVENT_DIR;
}
public synchronized void notifyCompletion(EventTaskReport report) {
@@ -156,25 +165,36 @@
submit(patterns);
}
- private Patterns initPattern(String eventsDir) {
- Nodeid nodeid = new Nodeid(new Value(null,
- EventDriver.CLIENT_NODE.getId()));
- List<Pattern> patternList = new ArrayList<Pattern>();
- String workingDir = cluster.getWorkingDir().getDir();
- String username = cluster.getUsername() == null ? System
- .getProperty("user.name") : cluster.getUsername();
- patternList.add(getDirectoryTransferPattern(username, eventsDir,
- nodeid, cluster.getMasterNode().getClusterIp(), workingDir));
+ private Patterns initPattern(String eventsDir) throws Exception {
+ Nodeid nodeid = new Nodeid(new Value(null, EventDriver.CLIENT_NODE.getId()));
+ List<Pattern> patternList = new ArrayList<Pattern>();
+ String workingDir = cluster.getWorkingDir().getDir();
+ String username = cluster.getUsername() == null ? System.getProperty("user.name") : cluster.getUsername();
+ patternList.add(getDirectoryTransferPattern(username, eventsDir, nodeid,
+ cluster.getMasterNode().getClusterIp(), workingDir));
- if (!cluster.getWorkingDir().isNFS()) {
- for (Node node : cluster.getNode()) {
- patternList.add(getDirectoryTransferPattern(username,
- eventsDir, nodeid, node.getClusterIp(), workingDir));
- }
- }
- Patterns patterns = new Patterns(patternList);
- return patterns;
- }
+ JAXBContext ctx = JAXBContext.newInstance(Configuration.class);
+ Marshaller marshaller = ctx.createMarshaller();
+ marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
+ String outputPathDir = System.getProperty("java.io.tmpdir") + File.separator + "conf-"
+ + System.getProperty("user.name");
+ new File(outputPathDir).mkdirs();
+ String outputPath = outputPathDir + File.separator + "configuration.xml";
+ marshaller.marshal(configuration, new FileOutputStream(outputPath));
+
+ patternList.add(getFileTransferPattern(username, outputPath, nodeid, cluster.getMasterNode().getClusterIp(),
+ workingDir));
+
+ if (!cluster.getWorkingDir().isNFS()) {
+ for (Node node : cluster.getNode()) {
+ patternList.add(getDirectoryTransferPattern(username, eventsDir, nodeid, node.getClusterIp(),
+ workingDir));
+
+ }
+ }
+ Patterns patterns = new Patterns(patternList);
+ return patterns;
+ }
private Pattern getDirectoryTransferPattern(String username, String src, Nodeid srcNode, String destNodeIp,
String destDir) {
@@ -183,16 +203,27 @@
return new Pattern(null, 1, null, event);
}
+ private Pattern getFileTransferPattern(String username, String src, Nodeid srcNode, String destNodeIp,
+ String destDir) {
+ String pargs = username + " " + src + " " + destNodeIp + " " + destDir;
+ Event event = new Event("file_transfer", srcNode, pargs);
+ return new Pattern(null, 1, null, event);
+ }
+
public IOutputHandler getErrorHandler() {
return outputHandler;
}
private Events initializeEvents() throws JAXBException, FileNotFoundException {
- File file = new File(eventsDir + File.separator + "events" + File.separator + "events.xml");
+ File file = new File(getEventsDir() + File.separator + "events.xml");
JAXBContext eventCtx = JAXBContext.newInstance(Events.class);
Unmarshaller unmarshaller = eventCtx.createUnmarshaller();
events = (Events) unmarshaller.unmarshal(file);
return events;
}
+ public String getEventsHomeDir() {
+ return eventsHomeDir;
+ }
+
}
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/DefaultOutputHandler.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/DefaultOutputHandler.java
index d05c5ab..f7155f3 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/DefaultOutputHandler.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/DefaultOutputHandler.java
@@ -16,6 +16,7 @@
import edu.uci.ics.asterix.event.schema.pattern.Event;
+
public class DefaultOutputHandler implements IOutputHandler {
@Override
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java
index 45499f4..82b6e9e 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java
@@ -23,11 +23,13 @@
import org.apache.commons.io.IOUtils;
+import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
import edu.uci.ics.asterix.event.driver.EventDriver;
import edu.uci.ics.asterix.event.schema.cluster.Cluster;
import edu.uci.ics.asterix.event.schema.cluster.Node;
import edu.uci.ics.asterix.event.schema.cluster.Property;
import edu.uci.ics.asterix.event.schema.pattern.Pattern;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
public class EventExecutor {
@@ -40,10 +42,11 @@
private static final String DAEMON = "DAEMON";
public void executeEvent(Node node, String script, List<String> args, boolean isDaemon, Cluster cluster,
- Pattern pattern, IOutputHandler outputHandler, EventrixClient client) throws IOException {
+ Pattern pattern, IOutputHandler outputHandler, AsterixEventServiceClient client) throws IOException {
List<String> pargs = new ArrayList<String>();
pargs.add("/bin/bash");
- pargs.add(client.getEventsDir() + File.separator + "scripts" + File.separator + EXECUTE_SCRIPT);
+ pargs.add(client.getEventsHomeDir() + File.separator + AsterixEventServiceUtil.EVENT_DIR + File.separator
+ + EXECUTE_SCRIPT);
StringBuffer envBuffer = new StringBuffer(IP_LOCATION + "=" + node.getClusterIp() + " ");
boolean isMasterNode = node.getId().equals(cluster.getMasterNode().getId());
@@ -60,6 +63,13 @@
if (javaOpts != null) {
builder.append(javaOpts);
}
+ if (node.getDebugPort() != null) {
+ int debugPort = node.getDebugPort().intValue();
+ if (!javaOpts.contains("-Xdebug")) {
+ builder.append((" "
+ + "-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=" + debugPort));
+ }
+ }
builder.append("\"");
envBuffer.append("JAVA_OPTS" + "=" + builder + " ");
}
@@ -71,6 +81,13 @@
if (javaOpts != null) {
builder.append(javaOpts);
}
+ if (node.getDebugPort() != null) {
+ int debugPort = node.getDebugPort().intValue();
+ if (!javaOpts.contains("-Xdebug")) {
+ builder.append((" "
+ + "-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=" + debugPort));
+ }
+ }
builder.append("\"");
envBuffer.append("JAVA_OPTS" + "=" + builder + " ");
}
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventTask.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventTask.java
index a764f9a..f98230b 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventTask.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventTask.java
@@ -20,6 +20,7 @@
import java.util.List;
import java.util.Timer;
import java.util.TimerTask;
+
import org.apache.log4j.Logger;
import edu.uci.ics.asterix.event.driver.EventDriver;
@@ -30,134 +31,124 @@
public class EventTask extends TimerTask {
- public static enum State {
- INITIALIZED, IN_PROGRESS, COMPLETED, FAILED
- }
+ public static enum State {
+ INITIALIZED,
+ IN_PROGRESS,
+ COMPLETED,
+ FAILED
+ }
- private static final Logger logger = Logger.getLogger(EventTask.class
- .getName());
+ private static final Logger logger = Logger.getLogger(EventTask.class.getName());
- private Pattern pattern;
- private Event event;
- private long interval = 0;
- private long initialDelay = 0;
- private int maxOccurs = Integer.MAX_VALUE;
- private int occurrenceCount = 0;
- private Timer timer;
- private String taskScript;
- private Node location;
- private List<String> taskArgs;
- private EventrixClient client;
- private List<Node> candidateLocations;
- private boolean dynamicLocation = false;
- private boolean reuseLocation = false;
- private State state;
+ private Pattern pattern;
+ private Event event;
+ private long interval = 0;
+ private long initialDelay = 0;
+ private int maxOccurs = Integer.MAX_VALUE;
+ private int occurrenceCount = 0;
+ private Timer timer;
+ private String taskScript;
+ private Node location;
+ private List<String> taskArgs;
+ private AsterixEventServiceClient client;
+ private List<Node> candidateLocations;
+ private boolean dynamicLocation = false;
+ private boolean reuseLocation = false;
+ private State state;
-
+ public EventTask(Pattern pattern, AsterixEventServiceClient client) {
+ this.pattern = pattern;
+ this.client = client;
+ Period period = pattern.getPeriod();
+ if (period != null && period.getAbsvalue() != null) {
+ this.interval = EventUtil.parseTimeInterval(period.getAbsvalue(), period.getUnit());
+ }
+ if (pattern.getDelay() != null) {
+ this.initialDelay = EventUtil.parseTimeInterval(new ValueType(pattern.getDelay().getValue()), pattern
+ .getDelay().getUnit());
+ }
+ if (pattern.getMaxOccurs() != null) {
+ this.maxOccurs = pattern.getMaxOccurs();
+ }
+ this.timer = new Timer();
+ taskArgs = EventUtil.getEventArgs(pattern);
+ candidateLocations = EventUtil.getCandidateLocations(pattern, client.getCluster());
+ if (pattern.getEvent().getNodeid().getValue().getRandom() != null && period != null && maxOccurs > 1) {
+ dynamicLocation = true;
+ reuseLocation = pattern.getEvent().getNodeid().getValue().getRandom().getRange().isReuse();
+ } else {
+ location = EventUtil.getEventLocation(pattern, candidateLocations, client.getCluster());
+ }
+ String scriptsDir;
+ if (location.getId().equals(EventDriver.CLIENT_NODE_ID)) {
+ scriptsDir = client.getEventsDir();
+ } else {
+ scriptsDir = client.getCluster().getWorkingDir().getDir() + File.separator + "events";
+ }
+ event = EventUtil.getEvent(pattern, client.getEvents());
+ taskScript = scriptsDir + File.separator + event.getScript();
+ state = State.INITIALIZED;
+ }
- public EventTask(Pattern pattern, EventrixClient client) {
- this.pattern = pattern;
- this.client = client;
- Period period = pattern.getPeriod();
- if (period != null && period.getAbsvalue() != null) {
- this.interval = EventUtil.parseTimeInterval(period.getAbsvalue(),
- period.getUnit());
- }
- if (pattern.getDelay() != null) {
- this.initialDelay = EventUtil.parseTimeInterval(new ValueType(
- pattern.getDelay().getValue()), pattern.getDelay()
- .getUnit());
- }
- if (pattern.getMaxOccurs() != null) {
- this.maxOccurs = pattern.getMaxOccurs();
- }
- this.timer = new Timer();
- taskArgs = EventUtil.getEventArgs(pattern);
- candidateLocations = EventUtil.getCandidateLocations(pattern,
- client.getCluster());
- if (pattern.getEvent().getNodeid().getValue().getRandom() != null
- && period != null && maxOccurs > 1) {
- dynamicLocation = true;
- reuseLocation = pattern.getEvent().getNodeid().getValue()
- .getRandom().getRange().isReuse();
- } else {
- location = EventUtil.getEventLocation(pattern, candidateLocations,
- client.getCluster());
- }
- String scriptsDir;
- if (location.getId().equals(EventDriver.CLIENT_NODE_ID)) {
- scriptsDir = client.getEventsDir() + File.separator + "events";
- } else {
- scriptsDir = client.getCluster().getWorkingDir().getDir()
- + File.separator + "eventrix" + File.separator + "events";
- }
- event = EventUtil.getEvent(pattern, client.getEvents());
- taskScript = scriptsDir + File.separator + event.getScript();
- state = State.INITIALIZED;
- }
+ public void start() {
+ if (interval > 0) {
+ timer.schedule(this, initialDelay, interval);
+ } else {
+ timer.schedule(this, initialDelay);
+ }
+ }
- public void start() {
- if (interval > 0) {
- timer.schedule(this, initialDelay, interval);
- } else {
- timer.schedule(this, initialDelay);
- }
- }
+ @Override
+ public void run() {
+ if (candidateLocations.size() == 0) {
+ timer.cancel();
+ client.notifyCompletion(new EventTaskReport(this));
+ } else {
+ if (dynamicLocation) {
+ location = EventUtil.getEventLocation(pattern, candidateLocations, client.getCluster());
+ if (!reuseLocation) {
+ candidateLocations.remove(location);
+ }
+ }
- @Override
- public void run() {
- if (candidateLocations.size() == 0) {
- timer.cancel();
- client.notifyCompletion(new EventTaskReport(this));
- } else {
- if (dynamicLocation) {
- location = EventUtil.getEventLocation(pattern,
- candidateLocations, client.getCluster());
- if (!reuseLocation) {
- candidateLocations.remove(location);
- }
- }
+ logger.debug(EventUtil.dateFormat.format(new Date()) + " " + "EVENT "
+ + pattern.getEvent().getType().toUpperCase() + " at " + location.getId().toUpperCase());
+ try {
+ if (!client.isDryRun()) {
+ new EventExecutor().executeEvent(location, taskScript, taskArgs, event.isDaemon(),
+ client.getCluster(), pattern, client.getErrorHandler(), client);
+ }
+ occurrenceCount++;
+ if (occurrenceCount >= maxOccurs) {
+ timer.cancel();
+ client.notifyCompletion(new EventTaskReport(this));
+ }
+ } catch (IOException ioe) {
+ timer.cancel();
+ client.notifyCompletion(new EventTaskReport(this, false, ioe));
+ }
+ }
- logger.debug(EventUtil.dateFormat.format(new Date()) + " "
- + "EVENT " + pattern.getEvent().getType().toUpperCase()
- + " at " + location.getId().toUpperCase());
- try {
- if (!client.isDryRun()) {
- new EventExecutor().executeEvent(location, taskScript,
- taskArgs, event.isDaemon(), client.getCluster(),
- pattern, client.getErrorHandler(), client);
- }
- occurrenceCount++;
- if (occurrenceCount >= maxOccurs) {
- timer.cancel();
- client.notifyCompletion(new EventTaskReport(this));
- }
- } catch (IOException ioe) {
- timer.cancel();
- client.notifyCompletion(new EventTaskReport(this, false, ioe));
- }
- }
+ }
- }
+ public Node getLocation() {
+ return location;
+ }
- public Node getLocation() {
- return location;
- }
+ public long getInterval() {
+ return interval;
+ }
- public long getInterval() {
- return interval;
- }
+ public long getInitialDelay() {
+ return initialDelay;
+ }
- public long getInitialDelay() {
- return initialDelay;
- }
+ public Pattern getPattern() {
+ return pattern;
+ }
- public Pattern getPattern() {
- return pattern;
- }
-
- public State getState() {
- return state;
- }
+ public State getState() {
+ return state;
+ }
}
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java
index 887e272..49b7abf 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java
@@ -16,7 +16,6 @@
import java.io.File;
import java.io.IOException;
-import java.math.BigInteger;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
@@ -36,252 +35,232 @@
public class EventUtil {
- public static final String EVENTS_DIR = "events";
- public static final String CLUSTER_CONF = "config/cluster.xml";
- public static final String PATTERN_CONF = "config/pattern.xml";
- public static final DateFormat dateFormat = new SimpleDateFormat(
- "yyyy/MM/dd HH:mm:ss");
- public static final String NC_JAVA_OPTS = "nc.java.opts";
- public static final String CC_JAVA_OPTS = "cc.java.opts";
+ public static final String EVENTS_DIR = "events";
+ public static final String CLUSTER_CONF = "config/cluster.xml";
+ public static final String PATTERN_CONF = "config/pattern.xml";
+ public static final DateFormat dateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss");
+ public static final String NC_JAVA_OPTS = "nc.java.opts";
+ public static final String CC_JAVA_OPTS = "cc.java.opts";
- private static final String IP_LOCATION = "IP_LOCATION";
- private static final String CLUSTER_ENV = "ENV";
- private static final String SCRIPT = "SCRIPT";
- private static final String ARGS = "ARGS";
- private static final String EXECUTE_SCRIPT = "events/execute.sh";
- private static final String LOCALHOST = "localhost";
- private static final String LOCALHOST_IP = "127.0.0.1";
+ private static final String IP_LOCATION = "IP_LOCATION";
+ private static final String CLUSTER_ENV = "ENV";
+ private static final String SCRIPT = "SCRIPT";
+ private static final String ARGS = "ARGS";
+ private static final String EXECUTE_SCRIPT = "events/execute.sh";
+ private static final String LOCALHOST = "localhost";
+ private static final String LOCALHOST_IP = "127.0.0.1";
- public static Cluster getCluster(String clusterConfigurationPath)
- throws JAXBException {
- File file = new File(clusterConfigurationPath);
- JAXBContext ctx = JAXBContext.newInstance(Cluster.class);
- Unmarshaller unmarshaller = ctx.createUnmarshaller();
- Cluster cluster = (Cluster) unmarshaller.unmarshal(file);
- if (cluster.getMasterNode().getClusterIp().equals(LOCALHOST)) {
- cluster.getMasterNode().setClusterIp(LOCALHOST_IP);
- }
- for (Node node : cluster.getNode()) {
- if (node.getClusterIp().equals(LOCALHOST)) {
- node.setClusterIp(LOCALHOST_IP);
- }
- }
- return cluster;
- }
+ public static Cluster getCluster(String clusterConfigurationPath) throws JAXBException {
+ File file = new File(clusterConfigurationPath);
+ JAXBContext ctx = JAXBContext.newInstance(Cluster.class);
+ Unmarshaller unmarshaller = ctx.createUnmarshaller();
+ Cluster cluster = (Cluster) unmarshaller.unmarshal(file);
+ if (cluster.getMasterNode().getClusterIp().equals(LOCALHOST)) {
+ cluster.getMasterNode().setClusterIp(LOCALHOST_IP);
+ }
+ for (Node node : cluster.getNode()) {
+ if (node.getClusterIp().equals(LOCALHOST)) {
+ node.setClusterIp(LOCALHOST_IP);
+ }
+ }
+ return cluster;
+ }
- public static long parseTimeInterval(ValueType v, String unit)
- throws IllegalArgumentException {
- int val = 0;
- switch (v.getType()) {
- case ABS:
- val = Integer.parseInt(v.getAbsoluteValue());
- break;
- case RANDOM_MIN_MAX:
- val = Randomizer.getInstance().getRandomInt(v.getMin(), v.getMax());
- break;
- case RANDOM_RANGE:
- String[] values = v.getRangeSet();
- val = Integer.parseInt(values[Randomizer.getInstance()
- .getRandomInt(0, values.length - 1)]);
- break;
- }
- return computeInterval(val, unit);
- }
+ public static long parseTimeInterval(ValueType v, String unit) throws IllegalArgumentException {
+ int val = 0;
+ switch (v.getType()) {
+ case ABS:
+ val = Integer.parseInt(v.getAbsoluteValue());
+ break;
+ case RANDOM_MIN_MAX:
+ val = Randomizer.getInstance().getRandomInt(v.getMin(), v.getMax());
+ break;
+ case RANDOM_RANGE:
+ String[] values = v.getRangeSet();
+ val = Integer.parseInt(values[Randomizer.getInstance().getRandomInt(0, values.length - 1)]);
+ break;
+ }
+ return computeInterval(val, unit);
+ }
- public static long parseTimeInterval(String v, String unit)
- throws IllegalArgumentException {
- int value = Integer.parseInt(v);
- return computeInterval(value, unit);
- }
+ public static long parseTimeInterval(String v, String unit) throws IllegalArgumentException {
+ int value = Integer.parseInt(v);
+ return computeInterval(value, unit);
+ }
- private static long computeInterval(int val, String unit) {
- int vmult = 1;
- if ("hr".equalsIgnoreCase(unit)) {
- vmult = 3600 * 1000;
- } else if ("min".equalsIgnoreCase(unit)) {
- vmult = 60 * 1000;
- } else if ("sec".equalsIgnoreCase(unit)) {
- vmult = 1000;
- } else
- throw new IllegalArgumentException(
- " invalid unit value specified for frequency (hr,min,sec)");
- return val * vmult;
+ private static long computeInterval(int val, String unit) {
+ int vmult = 1;
+ if ("hr".equalsIgnoreCase(unit)) {
+ vmult = 3600 * 1000;
+ } else if ("min".equalsIgnoreCase(unit)) {
+ vmult = 60 * 1000;
+ } else if ("sec".equalsIgnoreCase(unit)) {
+ vmult = 1000;
+ } else
+ throw new IllegalArgumentException(" invalid unit value specified for frequency (hr,min,sec)");
+ return val * vmult;
- }
+ }
- public static Event getEvent(Pattern pattern, Events events) {
- for (Event event : events.getEvent()) {
- if (event.getType().equals(pattern.getEvent().getType())) {
- return event;
- }
- }
- throw new IllegalArgumentException(" Unknown event type"
- + pattern.getEvent().getType());
- }
+ public static Event getEvent(Pattern pattern, Events events) {
+ for (Event event : events.getEvent()) {
+ if (event.getType().equals(pattern.getEvent().getType())) {
+ return event;
+ }
+ }
+ throw new IllegalArgumentException(" Unknown event type" + pattern.getEvent().getType());
+ }
- public static Node getEventLocation(Pattern pattern,
- List<Node> candidateLocations, Cluster cluster) {
- ValueType value = new ValueType(pattern.getEvent().getNodeid()
- .getValue());
- Node location = null;
- Type vtype = value.getType();
+ public static Node getEventLocation(Pattern pattern, List<Node> candidateLocations, Cluster cluster) {
+ ValueType value = new ValueType(pattern.getEvent().getNodeid().getValue());
+ Node location = null;
+ Type vtype = value.getType();
- switch (vtype) {
- case ABS:
- location = getNodeFromId(value.getAbsoluteValue(), cluster);
- break;
- case RANDOM_RANGE:
- int nodeIndex = Randomizer.getInstance().getRandomInt(0,
- candidateLocations.size() - 1);
- location = candidateLocations.get(nodeIndex);
- break;
- case RANDOM_MIN_MAX:
- throw new IllegalStateException(
- " Canont configure a min max value range for location");
- }
- return location;
+ switch (vtype) {
+ case ABS:
+ location = getNodeFromId(value.getAbsoluteValue(), cluster);
+ break;
+ case RANDOM_RANGE:
+ int nodeIndex = Randomizer.getInstance().getRandomInt(0, candidateLocations.size() - 1);
+ location = candidateLocations.get(nodeIndex);
+ break;
+ case RANDOM_MIN_MAX:
+ throw new IllegalStateException(" Canont configure a min max value range for location");
+ }
+ return location;
- }
+ }
- public static List<Node> getCandidateLocations(Pattern pattern,
- Cluster cluster) {
- ValueType value = new ValueType(pattern.getEvent().getNodeid()
- .getValue());
- List<Node> candidateList = new ArrayList<Node>();
- switch (value.getType()) {
- case ABS:
- candidateList.add(getNodeFromId(value.getAbsoluteValue(), cluster));
- break;
- case RANDOM_RANGE:
- boolean anyOption = false;
- String[] values = value.getRangeSet();
- for (String v : values) {
- if (v.equalsIgnoreCase("ANY")) {
- anyOption = true;
- }
- }
- if (anyOption) {
- for (Node node : cluster.getNode()) {
- candidateList.add(node);
- }
- } else {
- boolean found = false;
- for (String v : values) {
- for (Node node : cluster.getNode()) {
- if (node.getId().equals(v)) {
- candidateList.add(node);
- found = true;
- break;
- }
- }
- if (!found) {
- throw new IllegalStateException("Unknonw nodeId : " + v);
- }
- found = false;
- }
+ public static List<Node> getCandidateLocations(Pattern pattern, Cluster cluster) {
+ ValueType value = new ValueType(pattern.getEvent().getNodeid().getValue());
+ List<Node> candidateList = new ArrayList<Node>();
+ switch (value.getType()) {
+ case ABS:
+ candidateList.add(getNodeFromId(value.getAbsoluteValue(), cluster));
+ break;
+ case RANDOM_RANGE:
+ boolean anyOption = false;
+ String[] values = value.getRangeSet();
+ for (String v : values) {
+ if (v.equalsIgnoreCase("ANY")) {
+ anyOption = true;
+ }
+ }
+ if (anyOption) {
+ for (Node node : cluster.getNode()) {
+ candidateList.add(node);
+ }
+ } else {
+ boolean found = false;
+ for (String v : values) {
+ for (Node node : cluster.getNode()) {
+ if (node.getId().equals(v)) {
+ candidateList.add(node);
+ found = true;
+ break;
+ }
+ }
+ if (!found) {
+ throw new IllegalStateException("Unknonw nodeId : " + v);
+ }
+ found = false;
+ }
- }
- String[] excluded = value.getRangeExcluded();
- if (excluded != null && excluded.length > 0) {
- List<Node> markedForRemoval = new ArrayList<Node>();
- for (String exclusion : excluded) {
- for (Node node : candidateList) {
- if (node.getId().equals(exclusion)) {
- markedForRemoval.add(node);
- }
- }
- }
- candidateList.removeAll(markedForRemoval);
- }
- break;
- case RANDOM_MIN_MAX:
- throw new IllegalStateException(
- " Invalid value configured for location");
- }
- return candidateList;
- }
+ }
+ String[] excluded = value.getRangeExcluded();
+ if (excluded != null && excluded.length > 0) {
+ List<Node> markedForRemoval = new ArrayList<Node>();
+ for (String exclusion : excluded) {
+ for (Node node : candidateList) {
+ if (node.getId().equals(exclusion)) {
+ markedForRemoval.add(node);
+ }
+ }
+ }
+ candidateList.removeAll(markedForRemoval);
+ }
+ break;
+ case RANDOM_MIN_MAX:
+ throw new IllegalStateException(" Invalid value configured for location");
+ }
+ return candidateList;
+ }
- private static Node getNodeFromId(String nodeid, Cluster cluster) {
- if (nodeid.equals(EventDriver.CLIENT_NODE.getId())) {
- return EventDriver.CLIENT_NODE;
- }
+ private static Node getNodeFromId(String nodeid, Cluster cluster) {
+ if (nodeid.equals(EventDriver.CLIENT_NODE.getId())) {
+ return EventDriver.CLIENT_NODE;
+ }
- if (nodeid.equals(cluster.getMasterNode().getId())) {
- String logDir = cluster.getMasterNode().getLogDir() == null ? cluster
- .getLogDir()
- : cluster.getMasterNode().getLogDir();
- String javaHome = cluster.getMasterNode().getJavaHome() == null ? cluster
- .getJavaHome()
- : cluster.getMasterNode().getJavaHome();
- return new Node(cluster.getMasterNode().getId(), cluster
- .getMasterNode().getClusterIp(), javaHome, logDir, null,
- null, null);
- }
+ if (nodeid.equals(cluster.getMasterNode().getId())) {
+ String logDir = cluster.getMasterNode().getLogDir() == null ? cluster.getLogDir() : cluster.getMasterNode()
+ .getLogDir();
+ String javaHome = cluster.getMasterNode().getJavaHome() == null ? cluster.getJavaHome() : cluster
+ .getMasterNode().getJavaHome();
+ return new Node(cluster.getMasterNode().getId(), cluster.getMasterNode().getClusterIp(), javaHome, logDir,
+ null, null, null, cluster.getMasterNode().getDebugPort());
+ }
- List<Node> nodeList = cluster.getNode();
- for (Node node : nodeList) {
- if (node.getId().equals(nodeid)) {
- return node;
- }
- }
- StringBuffer buffer = new StringBuffer();
- buffer.append(EventDriver.CLIENT_NODE.getId() + ",");
- buffer.append(cluster.getMasterNode().getId() + ",");
- for (Node v : cluster.getNode()) {
- buffer.append(v.getId() + ",");
- }
- buffer.deleteCharAt(buffer.length() - 1);
- throw new IllegalArgumentException("Unknown node id :" + nodeid
- + " valid ids:" + buffer);
- }
+ List<Node> nodeList = cluster.getNode();
+ for (Node node : nodeList) {
+ if (node.getId().equals(nodeid)) {
+ return node;
+ }
+ }
+ StringBuffer buffer = new StringBuffer();
+ buffer.append(EventDriver.CLIENT_NODE.getId() + ",");
+ buffer.append(cluster.getMasterNode().getId() + ",");
+ for (Node v : cluster.getNode()) {
+ buffer.append(v.getId() + ",");
+ }
+ buffer.deleteCharAt(buffer.length() - 1);
+ throw new IllegalArgumentException("Unknown node id :" + nodeid + " valid ids:" + buffer);
+ }
- public static void executeEventScript(Node node, String script,
- List<String> args, Cluster cluster) throws IOException,
- InterruptedException {
- List<String> pargs = new ArrayList<String>();
- pargs.add("/bin/bash");
- pargs.add(EventDriver.getEventsDir() + "/" + EXECUTE_SCRIPT);
- StringBuffer argBuffer = new StringBuffer();
- String env = EventDriver.getStringifiedEnv(cluster) + " " + IP_LOCATION
- + "=" + node.getClusterIp();
- if (args != null) {
- for (String arg : args) {
- argBuffer.append(arg + " ");
- }
- }
- ProcessBuilder pb = new ProcessBuilder(pargs);
- pb.environment().putAll(EventDriver.getEnvironment());
- pb.environment().put(IP_LOCATION, node.getClusterIp());
- pb.environment().put(CLUSTER_ENV, env);
- pb.environment().put(SCRIPT, script);
- pb.environment().put(ARGS, argBuffer.toString());
- pb.start();
- }
+ public static void executeEventScript(Node node, String script, List<String> args, Cluster cluster)
+ throws IOException, InterruptedException {
+ List<String> pargs = new ArrayList<String>();
+ pargs.add("/bin/bash");
+ pargs.add(EventDriver.getEventsDir() + "/" + EXECUTE_SCRIPT);
+ StringBuffer argBuffer = new StringBuffer();
+ String env = EventDriver.getStringifiedEnv(cluster) + " " + IP_LOCATION + "=" + node.getClusterIp();
+ if (args != null) {
+ for (String arg : args) {
+ argBuffer.append(arg + " ");
+ }
+ }
+ ProcessBuilder pb = new ProcessBuilder(pargs);
+ pb.environment().putAll(EventDriver.getEnvironment());
+ pb.environment().put(IP_LOCATION, node.getClusterIp());
+ pb.environment().put(CLUSTER_ENV, env);
+ pb.environment().put(SCRIPT, script);
+ pb.environment().put(ARGS, argBuffer.toString());
+ pb.start();
+ }
- public static void executeLocalScript(Node node, String script,
- List<String> args) throws IOException, InterruptedException {
- List<String> pargs = new ArrayList<String>();
- pargs.add("/bin/bash");
- pargs.add(script);
- if (args != null) {
- pargs.addAll(args);
- }
- ProcessBuilder pb = new ProcessBuilder(pargs);
- pb.environment().putAll(EventDriver.getEnvironment());
- pb.environment().put(IP_LOCATION, node.getClusterIp());
- pb.start();
- }
+ public static void executeLocalScript(Node node, String script, List<String> args) throws IOException,
+ InterruptedException {
+ List<String> pargs = new ArrayList<String>();
+ pargs.add("/bin/bash");
+ pargs.add(script);
+ if (args != null) {
+ pargs.addAll(args);
+ }
+ ProcessBuilder pb = new ProcessBuilder(pargs);
+ pb.environment().putAll(EventDriver.getEnvironment());
+ pb.environment().put(IP_LOCATION, node.getClusterIp());
+ pb.start();
+ }
- public static List<String> getEventArgs(Pattern pattern) {
- List<String> pargs = new ArrayList<String>();
- if (pattern.getEvent().getPargs() == null) {
- return pargs;
- }
- String[] args = pattern.getEvent().getPargs().split(" ");
- for (String arg : args) {
- pargs.add(arg.trim());
- }
- return pargs;
- }
+ public static List<String> getEventArgs(Pattern pattern) {
+ List<String> pargs = new ArrayList<String>();
+ if (pattern.getEvent().getPargs() == null) {
+ return pargs;
+ }
+ String[] args = pattern.getEvent().getPargs().split(" ");
+ for (String arg : args) {
+ pargs.add(arg.trim());
+ }
+ return pargs;
+ }
}
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/IOutputHandler.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/IOutputHandler.java
index f6cf905..5dd370c 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/IOutputHandler.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/IOutputHandler.java
@@ -16,6 +16,7 @@
import edu.uci.ics.asterix.event.schema.pattern.Event;
+
public interface IOutputHandler {
public OutputAnalysis reportEventOutput(Event event, String output);
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/ValueType.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/ValueType.java
index b619002..2524d9a 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/ValueType.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/ValueType.java
@@ -16,6 +16,7 @@
import edu.uci.ics.asterix.event.schema.pattern.Value;
+
public class ValueType {
public static enum Type {
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixInstance.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/AsterixInstance.java
similarity index 98%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixInstance.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/model/AsterixInstance.java
index c9a4743..87f14b0 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixInstance.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/AsterixInstance.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.installer.model;
+package edu.uci.ics.asterix.event.model;
import java.io.Serializable;
import java.util.ArrayList;
@@ -30,7 +30,7 @@
private static final long serialVersionUID = 1L;
private static final int WEB_INTERFACE_PORT_DEFAULT = 19001;
-
+
public enum State {
ACTIVE,
INACTIVE,
@@ -61,7 +61,6 @@
this.asterixVersion = asterixVersion;
this.createdTimestamp = new Date();
this.backupInfo = new ArrayList<BackupInfo>();
-
}
public Date getModifiedTimestamp() {
@@ -212,4 +211,5 @@
public void setAsterixConfiguration(AsterixConfiguration asterixConfiguration) {
this.asterixConfiguration = asterixConfiguration;
}
+
}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixRuntimeState.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/AsterixRuntimeState.java
similarity index 93%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixRuntimeState.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/model/AsterixRuntimeState.java
index e1a5167..b8fbabb 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixRuntimeState.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/AsterixRuntimeState.java
@@ -12,13 +12,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.installer.model;
+package edu.uci.ics.asterix.event.model;
import java.io.Serializable;
import java.util.List;
public class AsterixRuntimeState implements Serializable {
+ private static final long serialVersionUID = 1L;
private final List<ProcessInfo> processes;
private final List<String> failedNCs;
private final boolean ccRunning;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/BackupInfo.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/BackupInfo.java
similarity index 97%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/BackupInfo.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/model/BackupInfo.java
index c88fddb..e1bcd8a 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/BackupInfo.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/BackupInfo.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.installer.model;
+package edu.uci.ics.asterix.event.model;
import java.io.Serializable;
import java.util.Date;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/EventList.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/EventList.java
similarity index 95%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/EventList.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/model/EventList.java
index 91592e7..fda814c 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/EventList.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/EventList.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.installer.model;
+package edu.uci.ics.asterix.event.model;
public class EventList {
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/ProcessInfo.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/ProcessInfo.java
similarity index 96%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/ProcessInfo.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/model/ProcessInfo.java
index 56dfc8a..4fde136 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/ProcessInfo.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/model/ProcessInfo.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.installer.model;
+package edu.uci.ics.asterix.event.model;
import java.io.Serializable;
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/AsterixEventService.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/AsterixEventService.java
new file mode 100644
index 0000000..6744746
--- /dev/null
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/AsterixEventService.java
@@ -0,0 +1,78 @@
+package edu.uci.ics.asterix.event.service;
+
+import java.io.File;
+import java.io.FileFilter;
+
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+import edu.uci.ics.asterix.event.error.OutputHandler;
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
+import edu.uci.ics.asterix.event.schema.cluster.Cluster;
+import edu.uci.ics.asterix.installer.schema.conf.Configuration;
+
+public class AsterixEventService {
+
+ private static final Logger LOGGER = Logger.getLogger(AsterixEventService.class.getName());
+ private static Configuration configuration;
+ private static String asterixDir;
+ private static String asterixZip;
+ private static String eventHome;
+
+ public static void initialize(Configuration configuration, String asterixDir, String eventHome) throws Exception {
+ AsterixEventService.configuration = configuration;
+ AsterixEventService.asterixDir = asterixDir;
+ AsterixEventService.asterixZip = initBinary("asterix-server");
+ AsterixEventService.eventHome = eventHome;
+ }
+
+ public static AsterixEventServiceClient getAsterixEventServiceClient(Cluster cluster, boolean transferArtifacts,
+ boolean dryRun) throws Exception {
+ AsterixEventServiceClient client = new AsterixEventServiceClient(configuration, eventHome, cluster,
+ transferArtifacts, dryRun, OutputHandler.INSTANCE);
+ return client;
+ }
+
+ public static AsterixEventServiceClient getAsterixEventServiceClient(Cluster cluster) throws Exception {
+ AsterixEventServiceClient client = new AsterixEventServiceClient(configuration, eventHome, cluster, false,
+ false, OutputHandler.INSTANCE);
+ return client;
+ }
+
+ private static String initBinary(final String fileNamePattern) {
+ File file = new File(asterixDir);
+ File[] zipFiles = file.listFiles(new FileFilter() {
+ public boolean accept(File arg0) {
+ return arg0.getAbsolutePath().contains(fileNamePattern) && arg0.isFile();
+ }
+ });
+ if (zipFiles.length == 0) {
+ String msg = " Binary not found at " + asterixDir;
+ LOGGER.log(Level.FATAL, msg);
+ throw new IllegalStateException(msg);
+ }
+ if (zipFiles.length > 1) {
+ String msg = " Multiple binaries found at " + asterixDir;
+ LOGGER.log(Level.FATAL, msg);
+ throw new IllegalStateException(msg);
+ }
+
+ return zipFiles[0].getAbsolutePath();
+ }
+
+ public static Configuration getConfiguration() {
+ return configuration;
+ }
+
+ public static String getAsterixZip() {
+ return asterixZip;
+ }
+
+ public static String getAsterixDir() {
+ return asterixDir;
+ }
+
+ public static String getEventHome() {
+ return eventHome;
+ }
+}
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/AsterixEventServiceUtil.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/AsterixEventServiceUtil.java
new file mode 100644
index 0000000..2a70862
--- /dev/null
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/AsterixEventServiceUtil.java
@@ -0,0 +1,551 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.event.service;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileFilter;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringWriter;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
+import java.util.jar.JarOutputStream;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+import java.util.zip.ZipOutputStream;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+
+import org.apache.commons.io.IOUtils;
+
+import edu.uci.ics.asterix.common.configuration.AsterixConfiguration;
+import edu.uci.ics.asterix.common.configuration.Coredump;
+import edu.uci.ics.asterix.common.configuration.Store;
+import edu.uci.ics.asterix.common.configuration.TransactionLogDir;
+import edu.uci.ics.asterix.event.driver.EventDriver;
+import edu.uci.ics.asterix.event.error.EventException;
+import edu.uci.ics.asterix.event.management.EventUtil;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.schema.cluster.Cluster;
+import edu.uci.ics.asterix.event.schema.cluster.Env;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
+import edu.uci.ics.asterix.event.schema.cluster.Property;
+
+public class AsterixEventServiceUtil {
+
+ public static final String TXN_LOG_DIR = "txnLogs";
+ public static final String TXN_LOG_DIR_KEY_SUFFIX = "txnLogDir";
+ public static final String ASTERIX_CONFIGURATION_FILE = "asterix-configuration.xml";
+ public static final String TXN_LOG_CONFIGURATION_FILE = "log.properties";
+ public static final String CLUSTER_CONFIGURATION_FILE = "cluster.xml";
+ public static final String ASTERIX_DIR = "asterix";
+ public static final String EVENT_DIR = "events";
+ public static final String DEFAULT_ASTERIX_CONFIGURATION_PATH = "conf" + File.separator + File.separator
+ + "asterix-configuration.xml";
+ public static final int CLUSTER_NET_PORT_DEFAULT = 1098;
+ public static final int CLIENT_NET_PORT_DEFAULT = 1099;
+ public static final int HTTP_PORT_DEFAULT = 8888;
+ public static final int WEB_INTERFACE_PORT_DEFAULT = 19001;
+
+ public static final String MANAGIX_INTERNAL_DIR = ".installer";
+ public static final String MANAGIX_CONF_XML = "conf" + File.separator + "managix-conf.xml";
+
+ public static AsterixInstance createAsterixInstance(String asterixInstanceName, Cluster cluster,
+ AsterixConfiguration asterixConfiguration) throws FileNotFoundException, IOException {
+ Node metadataNode = getMetadataNode(asterixInstanceName, cluster);
+ String asterixZipName = AsterixEventService.getAsterixZip().substring(
+ AsterixEventService.getAsterixZip().lastIndexOf(File.separator) + 1);
+ String asterixVersion = asterixZipName.substring("asterix-server-".length(),
+ asterixZipName.indexOf("-binary-assembly"));
+ AsterixInstance instance = new AsterixInstance(asterixInstanceName, cluster, asterixConfiguration,
+ metadataNode.getId(), asterixVersion);
+ return instance;
+ }
+
+ public static void createAsterixZip(AsterixInstance asterixInstance) throws IOException, InterruptedException,
+ JAXBException, EventException {
+
+ String modifiedZipPath = injectAsterixPropertyFile(AsterixEventService.getAsterixZip(), asterixInstance);
+ injectAsterixClusterConfigurationFile(modifiedZipPath, asterixInstance);
+ }
+
+ public static void createClusterProperties(Cluster cluster, AsterixConfiguration asterixConfiguration) {
+
+ String ccJavaOpts = null;
+ String ncJavaOpts = null;
+ for (edu.uci.ics.asterix.common.configuration.Property property : asterixConfiguration.getProperty()) {
+ if (property.getName().equalsIgnoreCase(EventUtil.CC_JAVA_OPTS)) {
+ ccJavaOpts = property.getValue();
+ } else if (property.getName().equalsIgnoreCase(EventUtil.NC_JAVA_OPTS)) {
+ ncJavaOpts = property.getValue();
+ }
+ }
+
+ poulateClusterEnvironmentProperties(cluster, ccJavaOpts, ncJavaOpts);
+ }
+
+ public static void poulateClusterEnvironmentProperties(Cluster cluster, String ccJavaOpts, String ncJavaOpts) {
+ List<Property> clusterProperties = null;
+ if (cluster.getEnv() != null && cluster.getEnv().getProperty() != null) {
+ clusterProperties = cluster.getEnv().getProperty();
+ clusterProperties.clear();
+ } else {
+ clusterProperties = new ArrayList<Property>();
+ }
+
+ clusterProperties.add(new Property(EventUtil.CC_JAVA_OPTS, ccJavaOpts));
+ clusterProperties.add(new Property(EventUtil.NC_JAVA_OPTS, ncJavaOpts));
+ clusterProperties.add(new Property("ASTERIX_HOME", cluster.getWorkingDir().getDir() + File.separator
+ + "asterix"));
+ clusterProperties.add(new Property("LOG_DIR", cluster.getLogDir()));
+ clusterProperties.add(new Property("JAVA_HOME", cluster.getJavaHome()));
+ clusterProperties.add(new Property("WORKING_DIR", cluster.getWorkingDir().getDir()));
+ clusterProperties.add(new Property("CLIENT_NET_IP", cluster.getMasterNode().getClientIp()));
+ clusterProperties.add(new Property("CLUSTER_NET_IP", cluster.getMasterNode().getClusterIp()));
+
+ int clusterNetPort = cluster.getMasterNode().getClusterPort() != null ? cluster.getMasterNode()
+ .getClusterPort().intValue() : CLUSTER_NET_PORT_DEFAULT;
+ int clientNetPort = cluster.getMasterNode().getClientPort() != null ? cluster.getMasterNode().getClientPort()
+ .intValue() : CLIENT_NET_PORT_DEFAULT;
+ int httpPort = cluster.getMasterNode().getHttpPort() != null ? cluster.getMasterNode().getHttpPort().intValue()
+ : HTTP_PORT_DEFAULT;
+
+ clusterProperties.add(new Property("CLIENT_NET_PORT", "" + clientNetPort));
+ clusterProperties.add(new Property("CLUSTER_NET_PORT", "" + clusterNetPort));
+ clusterProperties.add(new Property("HTTP_PORT", "" + httpPort));
+
+ cluster.setEnv(new Env(clusterProperties));
+ }
+
+ private static String injectAsterixPropertyFile(String origZipFile, AsterixInstance asterixInstance)
+ throws IOException, JAXBException {
+ writeAsterixConfigurationFile(asterixInstance);
+ String asterixInstanceDir = AsterixEventService.getAsterixDir() + File.separator + asterixInstance.getName();
+ unzip(origZipFile, asterixInstanceDir);
+ File sourceJar = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
+ + asterixInstance.getAsterixVersion() + ".jar");
+ File replacementFile = new File(asterixInstanceDir + File.separator + ASTERIX_CONFIGURATION_FILE);
+ replaceInJar(sourceJar, ASTERIX_CONFIGURATION_FILE, replacementFile);
+ new File(asterixInstanceDir + File.separator + ASTERIX_CONFIGURATION_FILE).delete();
+ String asterixZipName = AsterixEventService.getAsterixZip().substring(
+ AsterixEventService.getAsterixZip().lastIndexOf(File.separator) + 1);
+ zipDir(new File(asterixInstanceDir), new File(asterixInstanceDir + File.separator + asterixZipName));
+ return asterixInstanceDir + File.separator + asterixZipName;
+ }
+
+ private static String injectAsterixLogPropertyFile(String origZipFile, AsterixInstance asterixInstance)
+ throws IOException, EventException {
+ String asterixInstanceDir = AsterixEventService.getAsterixDir() + File.separator + asterixInstance.getName();
+ unzip(origZipFile, asterixInstanceDir);
+ File sourceJar1 = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
+ + asterixInstance.getAsterixVersion() + ".jar");
+ Properties txnLogProperties = new Properties();
+ URLClassLoader urlClassLoader = new URLClassLoader(new URL[] { sourceJar1.toURI().toURL() });
+ InputStream in = urlClassLoader.getResourceAsStream(TXN_LOG_CONFIGURATION_FILE);
+ if (in != null) {
+ txnLogProperties.load(in);
+ }
+
+ writeAsterixLogConfigurationFile(asterixInstance, txnLogProperties);
+
+ File sourceJar2 = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
+ + asterixInstance.getAsterixVersion() + ".jar");
+ File replacementFile = new File(asterixInstanceDir + File.separator + "log.properties");
+ replaceInJar(sourceJar2, TXN_LOG_CONFIGURATION_FILE, replacementFile);
+
+ new File(asterixInstanceDir + File.separator + "log.properties").delete();
+ String asterixZipName = AsterixEventService.getAsterixZip().substring(
+ AsterixEventService.getAsterixZip().lastIndexOf(File.separator) + 1);
+ zipDir(new File(asterixInstanceDir), new File(asterixInstanceDir + File.separator + asterixZipName));
+ return asterixInstanceDir + File.separator + asterixZipName;
+ }
+
+ private static String injectAsterixClusterConfigurationFile(String origZipFile, AsterixInstance asterixInstance)
+ throws IOException, EventException, JAXBException {
+ String asterixInstanceDir = AsterixEventService.getAsterixDir() + File.separator + asterixInstance.getName();
+ unzip(origZipFile, asterixInstanceDir);
+ File sourceJar = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
+ + asterixInstance.getAsterixVersion() + ".jar");
+ writeAsterixClusterConfigurationFile(asterixInstance);
+
+ File replacementFile = new File(asterixInstanceDir + File.separator + "cluster.xml");
+ replaceInJar(sourceJar, CLUSTER_CONFIGURATION_FILE, replacementFile);
+
+ new File(asterixInstanceDir + File.separator + CLUSTER_CONFIGURATION_FILE).delete();
+ String asterixZipName = AsterixEventService.getAsterixZip().substring(
+ AsterixEventService.getAsterixZip().lastIndexOf(File.separator) + 1);
+ zipDir(new File(asterixInstanceDir), new File(asterixInstanceDir + File.separator + asterixZipName));
+ return asterixInstanceDir + File.separator + asterixZipName;
+ }
+
+ private static void writeAsterixClusterConfigurationFile(AsterixInstance asterixInstance) throws IOException,
+ EventException, JAXBException {
+ String asterixInstanceName = asterixInstance.getName();
+ Cluster cluster = asterixInstance.getCluster();
+
+ JAXBContext ctx = JAXBContext.newInstance(Cluster.class);
+ Marshaller marshaller = ctx.createMarshaller();
+ marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
+ marshaller.marshal(cluster, new FileOutputStream(AsterixEventService.getAsterixDir() + File.separator
+ + asterixInstanceName + File.separator + "cluster.xml"));
+ }
+
+ public static void addLibraryToAsterixZip(AsterixInstance asterixInstance, String dataverseName,
+ String libraryName, String libraryPath) throws IOException {
+ File instanceDir = new File(AsterixEventService.getAsterixDir() + File.separator + asterixInstance.getName());
+ if (!instanceDir.exists()) {
+ instanceDir.mkdirs();
+ }
+ String asterixZipName = AsterixEventService.getAsterixZip().substring(
+ AsterixEventService.getAsterixZip().lastIndexOf(File.separator) + 1);
+
+ String sourceZip = instanceDir.getAbsolutePath() + File.separator + asterixZipName;
+ unzip(sourceZip, instanceDir.getAbsolutePath());
+ File libraryPathInZip = new File(instanceDir.getAbsolutePath() + File.separator + "external" + File.separator
+ + "library" + dataverseName + File.separator + "to-add" + File.separator + libraryName);
+ libraryPathInZip.mkdirs();
+ Runtime.getRuntime().exec("cp" + " " + libraryPath + " " + libraryPathInZip.getAbsolutePath());
+ Runtime.getRuntime().exec("rm " + sourceZip);
+ String destZip = AsterixEventService.getAsterixDir() + File.separator + asterixInstance.getName()
+ + File.separator + asterixZipName;
+ zipDir(instanceDir, new File(destZip));
+ Runtime.getRuntime().exec("mv" + " " + destZip + " " + sourceZip);
+ }
+
+ private static Node getMetadataNode(String asterixInstanceName, Cluster cluster) {
+ Node metadataNode = null;
+ if (cluster.getMetadataNode() != null) {
+ for(Node node: cluster.getNode()){
+ if(node.getId().equals(cluster.getMetadataNode())){
+ metadataNode = node;
+ break;
+ }
+ }
+ } else {
+ Random random = new Random();
+ int nNodes = cluster.getNode().size();
+ metadataNode = cluster.getNode().get(random.nextInt(nNodes));
+ }
+ return metadataNode;
+ }
+
+ public static String getNodeDirectories(String asterixInstanceName, Node node, Cluster cluster) {
+ String storeDataSubDir = asterixInstanceName + File.separator + "data" + File.separator;
+ String[] storeDirs = null;
+ StringBuffer nodeDataStore = new StringBuffer();
+ String storeDirValue = node.getStore();
+ if (storeDirValue == null) {
+ storeDirValue = cluster.getStore();
+ if (storeDirValue == null) {
+ throw new IllegalStateException(" Store not defined for node " + node.getId());
+ }
+ storeDataSubDir = node.getId() + File.separator + storeDataSubDir;
+ }
+
+ storeDirs = storeDirValue.split(",");
+ for (String ns : storeDirs) {
+ nodeDataStore.append(ns + File.separator + storeDataSubDir.trim());
+ nodeDataStore.append(",");
+ }
+ nodeDataStore.deleteCharAt(nodeDataStore.length() - 1);
+ return nodeDataStore.toString();
+ }
+
+ private static void writeAsterixConfigurationFile(AsterixInstance asterixInstance) throws IOException,
+ JAXBException {
+ String asterixInstanceName = asterixInstance.getName();
+ Cluster cluster = asterixInstance.getCluster();
+ String metadataNodeId = asterixInstance.getMetadataNodeId();
+
+ AsterixConfiguration configuration = asterixInstance.getAsterixConfiguration();
+ configuration.setInstanceName(asterixInstanceName);
+ configuration.setMetadataNode(asterixInstanceName + "_" + metadataNodeId);
+ String storeDir = null;
+ List<Store> stores = new ArrayList<Store>();
+ for (Node node : cluster.getNode()) {
+ storeDir = node.getStore() == null ? cluster.getStore() : node.getStore();
+ stores.add(new Store(asterixInstanceName + "_" + node.getId(), storeDir));
+ }
+ configuration.setStore(stores);
+
+ List<Coredump> coredump = new ArrayList<Coredump>();
+ String coredumpDir = null;
+ List<TransactionLogDir> txnLogDirs = new ArrayList<TransactionLogDir>();
+ String txnLogDir = null;
+ for (Node node : cluster.getNode()) {
+ coredumpDir = node.getLogDir() == null ? cluster.getLogDir() : node.getLogDir();
+ coredump.add(new Coredump(asterixInstanceName + "_" + node.getId(), coredumpDir + File.separator
+ + asterixInstanceName + "_" + node.getId()));
+
+ txnLogDir = node.getTxnLogDir() == null ? cluster.getTxnLogDir() : node.getTxnLogDir();
+ txnLogDirs.add(new TransactionLogDir(asterixInstanceName + "_" + node.getId(), txnLogDir));
+ }
+ configuration.setCoredump(coredump);
+ configuration.setTransactionLogDir(txnLogDirs);
+
+ File asterixConfDir = new File(AsterixEventService.getAsterixDir() + File.separator + asterixInstanceName);
+ asterixConfDir.mkdirs();
+
+ JAXBContext ctx = JAXBContext.newInstance(AsterixConfiguration.class);
+ Marshaller marshaller = ctx.createMarshaller();
+ marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
+ FileOutputStream os = new FileOutputStream(asterixConfDir + File.separator + ASTERIX_CONFIGURATION_FILE);
+ marshaller.marshal(configuration, os);
+ os.close();
+ }
+
+ private static void writeAsterixLogConfigurationFile(AsterixInstance asterixInstance, Properties logProperties)
+ throws IOException, EventException {
+ String asterixInstanceName = asterixInstance.getName();
+ Cluster cluster = asterixInstance.getCluster();
+ StringBuffer conf = new StringBuffer();
+ for (Map.Entry<Object, Object> p : logProperties.entrySet()) {
+ conf.append(p.getKey() + "=" + p.getValue() + "\n");
+ }
+
+ for (Node node : cluster.getNode()) {
+ String txnLogDir = node.getTxnLogDir() == null ? cluster.getTxnLogDir() : node.getTxnLogDir();
+ if (txnLogDir == null) {
+ throw new EventException("Transaction log directory (txn_log_dir) not configured for node: "
+ + node.getId());
+ }
+ conf.append(asterixInstanceName + "_" + node.getId() + "." + TXN_LOG_DIR_KEY_SUFFIX + "=" + txnLogDir
+ + "\n");
+ }
+ List<edu.uci.ics.asterix.common.configuration.Property> properties = asterixInstance.getAsterixConfiguration()
+ .getProperty();
+ for (edu.uci.ics.asterix.common.configuration.Property p : properties) {
+ if (p.getName().trim().toLowerCase().contains("log")) {
+ conf.append(p.getValue() + "=" + p.getValue());
+ }
+ }
+ dumpToFile(AsterixEventService.getAsterixDir() + File.separator + asterixInstanceName + File.separator
+ + "log.properties", conf.toString());
+
+ }
+
+ public static void unzip(String sourceFile, String destDir) throws IOException {
+ BufferedOutputStream dest = null;
+ FileInputStream fis = new FileInputStream(sourceFile);
+ ZipInputStream zis = new ZipInputStream(new BufferedInputStream(fis));
+ ZipEntry entry = null;
+
+ int BUFFER_SIZE = 4096;
+ while ((entry = zis.getNextEntry()) != null) {
+ String dst = destDir + File.separator + entry.getName();
+ if (entry.isDirectory()) {
+ createDir(destDir, entry);
+ continue;
+ }
+ int count;
+ byte data[] = new byte[BUFFER_SIZE];
+
+ // write the file to the disk
+ FileOutputStream fos = new FileOutputStream(dst);
+ dest = new BufferedOutputStream(fos, BUFFER_SIZE);
+ while ((count = zis.read(data, 0, BUFFER_SIZE)) != -1) {
+ dest.write(data, 0, count);
+ }
+ // close the output streams
+ dest.flush();
+ dest.close();
+ }
+
+ zis.close();
+ }
+
+ public static void zipDir(File sourceDir, File destFile) throws IOException {
+ FileOutputStream fos = new FileOutputStream(destFile);
+ ZipOutputStream zos = new ZipOutputStream(fos);
+ zipDir(sourceDir, destFile, zos);
+ zos.close();
+ }
+
+ private static void zipDir(File sourceDir, final File destFile, ZipOutputStream zos) throws IOException {
+ File[] dirList = sourceDir.listFiles(new FileFilter() {
+ public boolean accept(File f) {
+ return !f.getName().endsWith(destFile.getName());
+ }
+ });
+ for (int i = 0; i < dirList.length; i++) {
+ File f = dirList[i];
+ if (f.isDirectory()) {
+ zipDir(f, destFile, zos);
+ } else {
+ int bytesIn = 0;
+ byte[] readBuffer = new byte[2156];
+ FileInputStream fis = new FileInputStream(f);
+ ZipEntry entry = new ZipEntry(sourceDir.getName() + File.separator + f.getName());
+ zos.putNextEntry(entry);
+ while ((bytesIn = fis.read(readBuffer)) != -1) {
+ zos.write(readBuffer, 0, bytesIn);
+ }
+ fis.close();
+ }
+ }
+ }
+
+ private static void replaceInJar(File sourceJar, String origFile, File replacementFile) throws IOException {
+ String srcJarAbsPath = sourceJar.getAbsolutePath();
+ String srcJarSuffix = srcJarAbsPath.substring(srcJarAbsPath.lastIndexOf(File.separator) + 1);
+ String srcJarName = srcJarSuffix.split(".jar")[0];
+
+ String destJarName = srcJarName + "-managix";
+ String destJarSuffix = destJarName + ".jar";
+ File destJar = new File(sourceJar.getParentFile().getAbsolutePath() + File.separator + destJarSuffix);
+ // File destJar = new File(sourceJar.getAbsolutePath() + ".modified");
+ InputStream jarIs = null;
+ FileInputStream fis = new FileInputStream(replacementFile);
+ JarFile sourceJarFile = new JarFile(sourceJar);
+ Enumeration<JarEntry> entries = sourceJarFile.entries();
+ JarOutputStream jos = new JarOutputStream(new FileOutputStream(destJar));
+ byte[] buffer = new byte[2048];
+ int read;
+ while (entries.hasMoreElements()) {
+ JarEntry entry = (JarEntry) entries.nextElement();
+ String name = entry.getName();
+ if (name.equals(origFile)) {
+ continue;
+ }
+ jarIs = sourceJarFile.getInputStream(entry);
+ jos.putNextEntry(entry);
+ while ((read = jarIs.read(buffer)) != -1) {
+ jos.write(buffer, 0, read);
+ }
+ }
+ JarEntry entry = new JarEntry(origFile);
+ jos.putNextEntry(entry);
+ while ((read = fis.read(buffer)) != -1) {
+ jos.write(buffer, 0, read);
+ }
+ fis.close();
+ jos.close();
+ jarIs.close();
+ sourceJar.delete();
+ destJar.renameTo(sourceJar);
+ destJar.setExecutable(true);
+ }
+
+ public static void dumpToFile(String dest, String content) throws IOException {
+ FileWriter writer = new FileWriter(dest);
+ writer.write(content);
+ writer.close();
+ }
+
+ private static void createDir(String destDirectory, ZipEntry entry) {
+ String name = entry.getName();
+ int index = name.lastIndexOf(File.separator);
+ String dirSequence = name.substring(0, index);
+ File newDirs = new File(destDirectory + File.separator + dirSequence);
+ newDirs.mkdirs();
+ }
+
+ public static AsterixInstance validateAsterixInstanceExists(String name, State... permissibleStates)
+ throws Exception {
+ AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService().getAsterixInstance(name);
+ if (instance == null) {
+ throw new EventException("Asterix instance by name " + name + " does not exist.");
+ }
+ boolean valid = false;
+ for (State state : permissibleStates) {
+ if (state.equals(instance.getState())) {
+ valid = true;
+ break;
+ }
+ }
+ if (!valid) {
+ throw new EventException("Asterix instance by the name " + name + " is in " + instance.getState()
+ + " state ");
+ }
+ return instance;
+ }
+
+ public static void validateAsterixInstanceNotExists(String name) throws Exception {
+ AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService().getAsterixInstance(name);
+ if (instance != null) {
+ throw new EventException("Asterix instance by name " + name + " already exists.");
+ }
+ }
+
+ public static void evaluateConflictWithOtherInstances(AsterixInstance instance) throws Exception {
+ List<AsterixInstance> existingInstances = ServiceProvider.INSTANCE.getLookupService().getAsterixInstances();
+ List<String> usedIps = new ArrayList<String>();
+ String masterIp = instance.getCluster().getMasterNode().getClusterIp();
+ for (Node node : instance.getCluster().getNode()) {
+ usedIps.add(node.getClusterIp());
+ }
+ usedIps.add(instance.getCluster().getMasterNode().getClusterIp());
+ boolean conflictFound = false;
+ AsterixInstance conflictingInstance = null;
+ for (AsterixInstance existing : existingInstances) {
+ conflictFound = !existing.getState().equals(State.INACTIVE)
+ && existing.getCluster().getMasterNode().getClusterIp().equals(masterIp);
+ if (conflictFound) {
+ conflictingInstance = existing;
+ break;
+ }
+ for (Node n : existing.getCluster().getNode()) {
+ if (usedIps.contains(n.getClusterIp())) {
+ conflictFound = true;
+ conflictingInstance = existing;
+ break;
+ }
+ }
+ }
+ if (conflictFound) {
+ throw new Exception("Cluster definition conflicts with an existing instance of Asterix: "
+ + conflictingInstance.getName());
+ }
+ }
+
+ public static void deleteDirectory(String path) throws IOException {
+ Runtime.getRuntime().exec("rm -rf " + path);
+ }
+
+ public static String executeLocalScript(String path, List<String> args) throws Exception {
+ List<String> pargs = new ArrayList<String>();
+ pargs.add("/bin/bash");
+ pargs.add(path);
+ if (args != null) {
+ pargs.addAll(args);
+ }
+ ProcessBuilder pb = new ProcessBuilder(pargs);
+ pb.environment().putAll(EventDriver.getEnvironment());
+ pb.environment().put("IP_LOCATION", EventDriver.CLIENT_NODE.getClusterIp());
+ Process p = pb.start();
+ BufferedInputStream bis = new BufferedInputStream(p.getInputStream());
+ StringWriter writer = new StringWriter();
+ IOUtils.copy(bis, writer, "UTF-8");
+ return writer.toString();
+ }
+
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ILookupService.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/ILookupService.java
similarity index 92%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ILookupService.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/service/ILookupService.java
index 59fa198..cac504f 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ILookupService.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/ILookupService.java
@@ -12,11 +12,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.installer.service;
+package edu.uci.ics.asterix.event.service;
import java.util.List;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
import edu.uci.ics.asterix.installer.schema.conf.Configuration;
public interface ILookupService {
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/ServiceProvider.java
similarity index 94%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/service/ServiceProvider.java
index 98d72f4..56e5337 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/ServiceProvider.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.installer.service;
+package edu.uci.ics.asterix.event.service;
public class ServiceProvider {
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ZooKeeperService.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/ZooKeeperService.java
similarity index 91%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ZooKeeperService.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/service/ZooKeeperService.java
index d3304d2..5c059bb 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ZooKeeperService.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/service/ZooKeeperService.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.installer.service;
+package edu.uci.ics.asterix.event.service;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
@@ -34,10 +34,8 @@
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.data.Stat;
-import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.error.InstallerException;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
+import edu.uci.ics.asterix.event.error.EventException;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
import edu.uci.ics.asterix.installer.schema.conf.Configuration;
public class ZooKeeperService implements ILookupService {
@@ -47,8 +45,7 @@
private static final int ZOOKEEPER_LEADER_CONN_PORT = 2222;
private static final int ZOOKEEPER_LEADER_ELEC_PORT = 2223;
private static final int ZOOKEEPER_SESSION_TIME_OUT = 40 * 1000; //milliseconds
- private static final String ZOOKEEPER_HOME = InstallerDriver.getManagixHome() + File.separator
- + InstallerDriver.MANAGIX_INTERNAL_DIR + File.separator + "zookeeper";
+ private static final String ZOOKEEPER_HOME = AsterixEventService.getEventHome() + File.separator + "zookeeper";
private static final String ZOO_KEEPER_CONFIG = ZOOKEEPER_HOME + File.separator + "zk.cfg";
private boolean isRunning = false;
@@ -109,10 +106,10 @@
if (head == null) {
StringBuilder msg = new StringBuilder(
"Unable to start Zookeeper Service. This could be because of the following reasons.\n");
- msg.append("1) Managix is incorrectly configured. Please run " + InstallerDriver.getManagixHome()
- + "/bin/managix validate" + " to run a validation test and correct the errors reported.");
+ msg.append("1) Managix is incorrectly configured. Please run " + "managix validate"
+ + " to run a validation test and correct the errors reported.");
msg.append("\n2) If validation in (1) is successful, ensure that java_home parameter is set correctly in Managix configuration ("
- + InstallerDriver.getManagixHome() + File.separator + InstallerDriver.MANAGIX_CONF_XML + ")");
+ + AsterixEventServiceUtil.MANAGIX_CONF_XML + ")");
throw new Exception(msg.toString());
}
msgQ.take();
@@ -172,7 +169,7 @@
public void removeAsterixInstance(String name) throws Exception {
if (!exists(name)) {
- throw new InstallerException("Asterix instance by name " + name + " does not exists.");
+ throw new EventException("Asterix instance by name " + name + " does not exists.");
}
zk.delete(ASTERIX_INSTANCE_BASE_PATH + File.separator + name, DEFAULT_NODE_VERSION);
}
@@ -244,7 +241,7 @@
buffer.append("server" + "." + serverId + "=" + server + ":" + leaderConnPort + ":" + leaderElecPort + "\n");
serverId++;
}
- InstallerUtil.dumpToFile(zooKeeperConfigPath, buffer.toString());
+ AsterixEventServiceUtil.dumpToFile(zooKeeperConfigPath, buffer.toString());
}
}
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/util/AsterixConstants.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/util/AsterixConstants.java
new file mode 100644
index 0000000..ff600ce
--- /dev/null
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/util/AsterixConstants.java
@@ -0,0 +1,7 @@
+package edu.uci.ics.asterix.event.util;
+
+public class AsterixConstants {
+
+ public static String ASTERIX_ROOT_METADATA_DIR = "asterix_root_metadata";
+
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/events/PatternCreator.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/util/PatternCreator.java
similarity index 72%
rename from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/events/PatternCreator.java
rename to asterix-events/src/main/java/edu/uci/ics/asterix/event/util/PatternCreator.java
index fe61462..e06d66c 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/events/PatternCreator.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/util/PatternCreator.java
@@ -12,17 +12,22 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.installer.events;
+package edu.uci.ics.asterix.event.util;
import java.io.File;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
+import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import edu.uci.ics.asterix.event.driver.EventDriver;
+import edu.uci.ics.asterix.event.error.VerificationUtil;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.BackupInfo;
+import edu.uci.ics.asterix.event.model.BackupInfo.BackupType;
import edu.uci.ics.asterix.event.schema.cluster.Cluster;
import edu.uci.ics.asterix.event.schema.cluster.Node;
import edu.uci.ics.asterix.event.schema.pattern.Delay;
@@ -31,20 +36,20 @@
import edu.uci.ics.asterix.event.schema.pattern.Pattern;
import edu.uci.ics.asterix.event.schema.pattern.Patterns;
import edu.uci.ics.asterix.event.schema.pattern.Value;
-import edu.uci.ics.asterix.installer.command.BackupCommand;
-import edu.uci.ics.asterix.installer.command.StopCommand;
-import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.error.VerificationUtil;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.BackupInfo;
-import edu.uci.ics.asterix.installer.model.BackupInfo.BackupType;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ILookupService;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
import edu.uci.ics.asterix.installer.schema.conf.Backup;
-import edu.uci.ics.asterix.installer.service.ILookupService;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
public class PatternCreator {
+ public static PatternCreator INSTANCE = new PatternCreator();
+
+ private PatternCreator() {
+
+ }
+
private ILookupService lookupService = ServiceProvider.INSTANCE.getLookupService();
private void addInitialDelay(Pattern p, int delay, String unit) {
@@ -94,9 +99,9 @@
return patterns;
}
- public Patterns getStopCommandPattern(StopCommand stopCommand) throws Exception {
+ public Patterns getStopCommandPattern(String asterixInstanceName) throws Exception {
List<Pattern> ps = new ArrayList<Pattern>();
- AsterixInstance asterixInstance = lookupService.getAsterixInstance(stopCommand.getAsterixInstanceName());
+ AsterixInstance asterixInstance = lookupService.getAsterixInstance(asterixInstanceName);
Cluster cluster = asterixInstance.getCluster();
String ccLocation = cluster.getMasterNode().getId();
@@ -104,7 +109,6 @@
addInitialDelay(createCC, 5, "sec");
ps.add(createCC);
- String asterixInstanceName = stopCommand.getAsterixInstanceName();
int nodeControllerIndex = 1;
for (Node node : cluster.getNode()) {
Pattern createNC = createNCStopPattern(node.getId(), asterixInstanceName + "_" + nodeControllerIndex);
@@ -161,8 +165,9 @@
iodevices = node.getIodevices() == null ? instance.getCluster().getIodevices() : node.getIodevices();
store = node.getStore() == null ? cluster.getStore() : node.getStore();
pargs = workingDir + " " + instance.getName() + " " + iodevices + " " + store + " "
- + BackupCommand.ASTERIX_ROOT_METADATA_DIR + " " + InstallerUtil.TXN_LOG_DIR + " " + backupId + " "
- + hdfsBackupDir + " " + "hdfs" + " " + node.getId() + " " + hdfsUrl + " " + hadoopVersion;
+ + AsterixConstants.ASTERIX_ROOT_METADATA_DIR + " " + AsterixEventServiceUtil.TXN_LOG_DIR + " "
+ + backupId + " " + hdfsBackupDir + " " + "hdfs" + " " + node.getId() + " " + hdfsUrl + " "
+ + hadoopVersion;
Event event = new Event("backup", nodeid, pargs);
patternList.add(new Pattern(null, 1, null, event));
}
@@ -185,7 +190,7 @@
txnLogDir = node.getTxnLogDir() == null ? instance.getCluster().getTxnLogDir() : node.getTxnLogDir();
store = node.getStore() == null ? cluster.getStore() : node.getStore();
pargs = workingDir + " " + instance.getName() + " " + iodevices + " " + store + " "
- + BackupCommand.ASTERIX_ROOT_METADATA_DIR + " " + txnLogDir + " " + backupId + " " + backupDir
+ + AsterixConstants.ASTERIX_ROOT_METADATA_DIR + " " + txnLogDir + " " + backupId + " " + backupDir
+ " " + "local" + " " + node.getId();
Event event = new Event("backup", nodeid, pargs);
patternList.add(new Pattern(null, 1, null, event));
@@ -210,8 +215,9 @@
String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
nodeStore = node.getStore() == null ? clusterStore : node.getStore();
pargs = workingDir + " " + instance.getName() + " " + iodevices + " " + nodeStore + " "
- + BackupCommand.ASTERIX_ROOT_METADATA_DIR + " " + InstallerUtil.TXN_LOG_DIR + " " + backupId + " "
- + " " + hdfsBackupDir + " " + "hdfs" + " " + node.getId() + " " + hdfsUrl + " " + hadoopVersion;
+ + AsterixConstants.ASTERIX_ROOT_METADATA_DIR + " " + AsterixEventServiceUtil.TXN_LOG_DIR + " "
+ + backupId + " " + " " + hdfsBackupDir + " " + "hdfs" + " " + node.getId() + " " + hdfsUrl + " "
+ + hadoopVersion;
Event event = new Event("restore", nodeid, pargs);
patternList.add(new Pattern(null, 1, null, event));
}
@@ -232,8 +238,8 @@
String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
nodeStore = node.getStore() == null ? clusterStore : node.getStore();
pargs = workingDir + " " + instance.getName() + " " + iodevices + " " + nodeStore + " "
- + BackupCommand.ASTERIX_ROOT_METADATA_DIR + " " + InstallerUtil.TXN_LOG_DIR + " " + backupId + " "
- + backupDir + " " + "local" + " " + node.getId();
+ + AsterixConstants.ASTERIX_ROOT_METADATA_DIR + " " + AsterixEventServiceUtil.TXN_LOG_DIR + " "
+ + backupId + " " + backupDir + " " + "local" + " " + node.getId();
Event event = new Event("restore", nodeid, pargs);
patternList.add(new Pattern(null, 1, null, event));
}
@@ -243,9 +249,8 @@
public Patterns createHadoopLibraryTransferPattern(Cluster cluster) throws Exception {
List<Pattern> patternList = new ArrayList<Pattern>();
String workingDir = cluster.getWorkingDir().getDir();
- String hadoopVersion = InstallerDriver.getConfiguration().getBackup().getHdfs().getVersion();
- File hadoopDir = new File(InstallerDriver.getManagixHome() + File.separator
- + InstallerDriver.MANAGIX_INTERNAL_DIR + File.separator + "hadoop-" + hadoopVersion);
+ String hadoopVersion = AsterixEventService.getConfiguration().getBackup().getHdfs().getVersion();
+ File hadoopDir = new File(AsterixEventService.getEventHome() + File.separator + "hadoop-" + hadoopVersion);
if (!hadoopDir.exists()) {
throw new IllegalStateException("Hadoop version :" + hadoopVersion + " not supported");
}
@@ -332,8 +337,8 @@
private Patterns createRemoveHDFSBackupPattern(AsterixInstance instance, String hdfsBackupDir) throws Exception {
List<Pattern> patternList = new ArrayList<Pattern>();
Cluster cluster = instance.getCluster();
- String hdfsUrl = InstallerDriver.getConfiguration().getBackup().getHdfs().getUrl();
- String hadoopVersion = InstallerDriver.getConfiguration().getBackup().getHdfs().getVersion();
+ String hdfsUrl = AsterixEventService.getConfiguration().getBackup().getHdfs().getUrl();
+ String hadoopVersion = AsterixEventService.getConfiguration().getBackup().getHdfs().getVersion();
String workingDir = cluster.getWorkingDir().getDir();
Node launchingNode = cluster.getNode().get(0);
Nodeid nodeid = new Nodeid(new Value(null, launchingNode.getId()));
@@ -386,6 +391,85 @@
return patterns;
}
+ public Patterns getLibraryInstallPattern(AsterixInstance instance, String dataverse, String libraryName,
+ String libraryPath) throws Exception {
+ List<Pattern> patternList = new ArrayList<Pattern>();
+ Cluster cluster = instance.getCluster();
+ Nodeid nodeid = new Nodeid(new Value(null, EventDriver.CLIENT_NODE.getId()));
+ String username = cluster.getUsername() != null ? cluster.getUsername() : System.getProperty("user.name");
+ String workingDir = cluster.getWorkingDir().getDir();
+ String destDir = workingDir + File.separator + "library" + File.separator + dataverse + File.separator
+ + libraryName;
+ String fileToTransfer = new File(libraryPath).getAbsolutePath();
+
+ Iterator<Node> installTargets = cluster.getNode().iterator();
+ Node installNode = installTargets.next();
+ String destinationIp = installNode.getClusterIp();
+ String pargs = username + " " + fileToTransfer + " " + destinationIp + " " + destDir + " " + "unpack";
+ Event event = new Event("file_transfer", nodeid, pargs);
+ Pattern p = new Pattern(null, 1, null, event);
+ patternList.add(p);
+
+ if (!cluster.getWorkingDir().isNFS()) {
+ while (installTargets.hasNext()) {
+ Node node = installTargets.next();
+ pargs = username + " " + fileToTransfer + " " + node.getClusterIp() + " " + destDir + " " + "unpack";
+ event = new Event("file_transfer", nodeid, pargs);
+ p = new Pattern(null, 1, null, event);
+ patternList.add(p);
+ }
+
+ pargs = username + " " + fileToTransfer + " " + cluster.getMasterNode().getClusterIp() + " " + destDir
+ + " " + "unpack";
+ event = new Event("file_transfer", nodeid, pargs);
+ p = new Pattern(null, 1, null, event);
+ patternList.add(p);
+ }
+ return new Patterns(patternList);
+ }
+
+ public Patterns getLibraryUninstallPattern(AsterixInstance instance, String dataverse, String libraryName)
+ throws Exception {
+ List<Pattern> patternList = new ArrayList<Pattern>();
+ Cluster cluster = instance.getCluster();
+ String workingDir = cluster.getWorkingDir().getDir();
+ String destFile = dataverse + "." + libraryName;
+ String pargs = workingDir + File.separator + "uninstall" + " " + destFile;
+
+ String metadataNodeId = instance.getMetadataNodeId();
+ Nodeid nodeid = new Nodeid(new Value(null, metadataNodeId));
+ Event event = new Event("file_create", nodeid, pargs);
+ Pattern p = new Pattern(null, 1, null, event);
+ patternList.add(p);
+
+ Iterator<Node> uninstallTargets = cluster.getNode().iterator();
+ String libDir = workingDir + File.separator + "library" + File.separator + dataverse + File.separator
+ + libraryName;
+ Node uninstallNode = uninstallTargets.next();
+ nodeid = new Nodeid(new Value(null, uninstallNode.getId()));
+ event = new Event("file_delete", nodeid, libDir);
+ p = new Pattern(null, 1, null, event);
+ patternList.add(p);
+ pargs = libDir;
+
+ if (!cluster.getWorkingDir().isNFS()) {
+ while (uninstallTargets.hasNext()) {
+ uninstallNode = uninstallTargets.next();
+ nodeid = new Nodeid(new Value(null, uninstallNode.getId()));
+ event = new Event("file_delete", nodeid, pargs);
+ p = new Pattern(null, 1, null, event);
+ patternList.add(p);
+ }
+
+ nodeid = new Nodeid(new Value(null, cluster.getMasterNode().getId()));
+ event = new Event("file_delete", nodeid, pargs);
+ p = new Pattern(null, 1, null, event);
+ patternList.add(p);
+
+ }
+ return new Patterns(patternList);
+ }
+
private Patterns createRemoveAsterixRootMetadata(AsterixInstance instance) throws Exception {
List<Pattern> patternList = new ArrayList<Pattern>();
Cluster cluster = instance.getCluster();
@@ -395,7 +479,7 @@
for (Node node : cluster.getNode()) {
String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
String primaryIODevice = iodevices.split(",")[0].trim();
- pargs = primaryIODevice + File.separator + BackupCommand.ASTERIX_ROOT_METADATA_DIR;
+ pargs = primaryIODevice + File.separator + AsterixConstants.ASTERIX_ROOT_METADATA_DIR;
nodeid = new Nodeid(new Value(null, node.getId()));
event = new Event("file_delete", nodeid, pargs);
patternList.add(new Pattern(null, 1, null, event));
@@ -447,9 +531,9 @@
private Pattern createCopyHyracksPattern(String instanceName, Cluster cluster, String destinationIp, String destDir) {
Nodeid nodeid = new Nodeid(new Value(null, EventDriver.CLIENT_NODE.getId()));
String username = cluster.getUsername() != null ? cluster.getUsername() : System.getProperty("user.name");
- String asterixZipName = InstallerDriver.getAsterixZip().substring(
- InstallerDriver.getAsterixZip().lastIndexOf(File.separator) + 1);
- String fileToTransfer = new File(InstallerDriver.getAsterixDir() + File.separator + instanceName
+ String asterixZipName = AsterixEventService.getAsterixZip().substring(
+ AsterixEventService.getAsterixZip().lastIndexOf(File.separator) + 1);
+ String fileToTransfer = new File(AsterixEventService.getAsterixDir() + File.separator + instanceName
+ File.separator + asterixZipName).getAbsolutePath();
String pargs = username + " " + fileToTransfer + " " + destinationIp + " " + destDir + " " + "unpack";
Event event = new Event("file_transfer", nodeid, pargs);
@@ -481,10 +565,48 @@
return new Pattern(null, 1, null, event);
}
+ public Patterns createPrepareNodePattern(String instanceName, Cluster cluster, Node nodeToBeAdded) {
+ List<Pattern> ps = new ArrayList<Pattern>();
+ boolean workingDirOnNFS = cluster.getWorkingDir().isNFS();
+ if (!workingDirOnNFS) {
+ String ccLocationIp = cluster.getMasterNode().getClusterIp();
+ String destDir = cluster.getWorkingDir().getDir() + File.separator + "asterix";
+ Pattern copyHyracks = createCopyHyracksPattern(instanceName, cluster, ccLocationIp, destDir);
+ ps.add(copyHyracks);
+
+ String workingDir = cluster.getWorkingDir().getDir();
+ String hadoopVersion = AsterixEventService.getConfiguration().getBackup().getHdfs().getVersion();
+ File hadoopDir = new File(AsterixEventService.getEventHome() + File.separator + "hadoop-" + hadoopVersion);
+ if (!hadoopDir.exists()) {
+ throw new IllegalStateException("Hadoop version :" + hadoopVersion + " not supported");
+ }
+
+ Nodeid nodeid = new Nodeid(new Value(null, EventDriver.CLIENT_NODE.getId()));
+ String username = cluster.getUsername() != null ? cluster.getUsername() : System.getProperty("user.name");
+ String pargs = username + " " + hadoopDir.getAbsolutePath() + " " + cluster.getMasterNode().getClusterIp()
+ + " " + workingDir;
+ Event event = new Event("directory_transfer", nodeid, pargs);
+ Pattern p = new Pattern(null, 1, null, event);
+ addInitialDelay(p, 2, "sec");
+ ps.add(p);
+
+ nodeid = new Nodeid(new Value(null, nodeToBeAdded.getId()));
+ pargs = cluster.getUsername() + " " + hadoopDir.getAbsolutePath() + " " + nodeToBeAdded.getClusterIp()
+ + " " + workingDir;
+ event = new Event("directory_transfer", nodeid, pargs);
+ p = new Pattern(null, 1, null, event);
+ addInitialDelay(p, 2, "sec");
+ ps.add(p);
+ }
+
+ Patterns patterns = new Patterns(ps);
+ return patterns;
+ }
+
public Patterns getGenerateLogPattern(String asterixInstanceName, Cluster cluster, String outputDir) {
List<Pattern> patternList = new ArrayList<Pattern>();
- Map<String,String> nodeLogs = new HashMap<String,String>();
-
+ Map<String, String> nodeLogs = new HashMap<String, String>();
+
String username = cluster.getUsername() == null ? System.getProperty("user.name") : cluster.getUsername();
String srcHost = cluster.getMasterNode().getClientIp();
Nodeid nodeid = new Nodeid(new Value(null, EventDriver.CLIENT_NODE.getId()));
@@ -495,15 +617,15 @@
Event event = new Event("directory_copy", nodeid, pargs);
Pattern p = new Pattern(null, 1, null, event);
patternList.add(p);
- nodeLogs.put(cluster.getMasterNode().getClusterIp(),srcDir);
+ nodeLogs.put(cluster.getMasterNode().getClusterIp(), srcDir);
for (Node node : cluster.getNode()) {
srcHost = node.getClusterIp();
srcDir = node.getLogDir() == null ? cluster.getLogDir() : node.getLogDir();
- if(nodeLogs.get(node.getClusterIp()) != null && nodeLogs.get(node.getClusterIp()).equals(srcDir)){
+ if (nodeLogs.get(node.getClusterIp()) != null && nodeLogs.get(node.getClusterIp()).equals(srcDir)) {
continue;
}
destDir = outputDir + File.separator + node.getId();
- pargs = username + " " + srcHost + " " + srcDir + " " + destDir;
+ pargs = username + " " + srcHost + " " + srcDir + " " + destDir;
event = new Event("directory_copy", nodeid, pargs);
p = new Pattern(null, 1, null, event);
patternList.add(p);
diff --git a/asterix-events/src/main/resources/events/backup/backup.sh b/asterix-events/src/main/resources/events/backup/backup.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/events/cc_failure/cc_failure.sh b/asterix-events/src/main/resources/events/cc_failure/cc_failure.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/events/cc_start/cc_start.sh b/asterix-events/src/main/resources/events/cc_start/cc_start.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/events/events.xml b/asterix-events/src/main/resources/events/events.xml
index 01495cb..67a29a2 100644
--- a/asterix-events/src/main/resources/events/events.xml
+++ b/asterix-events/src/main/resources/events/events.xml
@@ -94,7 +94,14 @@
<type>file_delete</type>
<script>file/delete.sh</script>
<description>Deletes a file on the local file system to a remote node</description>
- <args>local_source_path destination_node destination_path</args>
+ <args>destination_node destination_path</args>
+ <daemon>false</daemon>
+ </event>
+ <event>
+ <type>file_create</type>
+ <script>file/create_file.sh</script>
+ <description>Creates a file on the local file system to a remote node</description>
+ <args>destination_node destination_path</args>
<daemon>false</daemon>
</event>
<event>
diff --git a/asterix-events/src/main/resources/events/execute.sh b/asterix-events/src/main/resources/events/execute.sh
new file mode 100644
index 0000000..30f6c2f
--- /dev/null
+++ b/asterix-events/src/main/resources/events/execute.sh
@@ -0,0 +1,27 @@
+USERNAME=$1
+if [ ! -d $MANAGIX_HOME/logs ];
+then
+ mkdir -p $MANAGIX_HOME/logs
+fi
+LOGDIR=$MANAGIX_HOME/logs
+if [ $DAEMON == "false" ]; then
+ if [ -z $USERNAME ]
+ then
+ cmd_output=$(ssh $IP_LOCATION "$ENV $SCRIPT $ARGS" 2>&1 >/dev/null)
+ echo "ssh $IP_LOCATION $ENV $SCRIPT $ARGS" >> $LOGDIR/execute.log
+ echo "$cmd_output"
+ else
+ echo "ssh -l $USERNAME $IP_LOCATION $ENV $SCRIPT $ARGS" >> $LOGDIR/execute.log
+ cmd_output=$(ssh -l $USERNAME $IP_LOCATION "$ENV $SCRIPT $ARGS" 2>&1 >/dev/null)
+ echo "$cmd_output"
+ fi
+else
+ if [ -z $USERNAME ];
+ then
+ echo "ssh $IP_LOCATION $ENV $SCRIPT $ARGS &" >> $LOGDIR/execute.log
+ ssh $IP_LOCATION "$ENV $SCRIPT $ARGS" &
+ else
+ echo "ssh -l $USERNAME $IP_LOCATION $ENV $SCRIPT $ARGS &" >> $LOGDIR/execute.log
+ ssh -l $USERNAME $IP_LOCATION "$ENV $SCRIPT $ARGS" &
+ fi
+fi
diff --git a/asterix-events/src/main/resources/events/file/create_file.sh b/asterix-events/src/main/resources/events/file/create_file.sh
new file mode 100644
index 0000000..762a2d3
--- /dev/null
+++ b/asterix-events/src/main/resources/events/file/create_file.sh
@@ -0,0 +1,3 @@
+mkdir -p $1
+echo "touch $1/$2" >> ~/file_create.log
+touch $1/$2
diff --git a/asterix-events/src/main/resources/events/file/delete.sh b/asterix-events/src/main/resources/events/file/delete.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/events/file/dir_transfer.sh b/asterix-events/src/main/resources/events/file/dir_transfer.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/events/file/transfer.sh b/asterix-events/src/main/resources/events/file/transfer.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/events/hdfs/delete.sh b/asterix-events/src/main/resources/events/hdfs/delete.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/events/node_failure/nc_failure.sh b/asterix-events/src/main/resources/events/node_failure/nc_failure.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/events/node_info/node_info.sh b/asterix-events/src/main/resources/events/node_info/node_info.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/events/node_join/nc_join.sh b/asterix-events/src/main/resources/events/node_join/nc_join.sh
old mode 100755
new mode 100644
index e0254c9..a18fe09
--- a/asterix-events/src/main/resources/events/node_join/nc_join.sh
+++ b/asterix-events/src/main/resources/events/node_join/nc_join.sh
@@ -19,5 +19,7 @@
then
mkdir -p $LOG_DIR
fi
+
cd $WORKING_DIR
+
$ASTERIX_HOME/bin/asterixnc -node-id $NC_ID -cc-host $CC_HOST -cc-port $CLUSTER_NET_PORT -cluster-net-ip-address $IP_LOCATION -data-ip-address $IP_LOCATION -iodevices $IO_DEVICES -result-ip-address $IP_LOCATION &> $LOG_DIR/${NC_ID}.log
diff --git a/asterix-events/src/main/resources/events/node_restart/nc_restart.sh b/asterix-events/src/main/resources/events/node_restart/nc_restart.sh
old mode 100755
new mode 100644
diff --git a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.1.ddl.aql b/asterix-events/src/main/resources/events/prepare.sh
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.1.ddl.aql
rename to asterix-events/src/main/resources/events/prepare.sh
diff --git a/asterix-events/src/main/resources/events/restore/restore.sh b/asterix-events/src/main/resources/events/restore/restore.sh
old mode 100755
new mode 100644
diff --git a/asterix-installer/src/main/resources/schema/installer-conf.xsd b/asterix-events/src/main/resources/schema/installer-conf.xsd
similarity index 100%
rename from asterix-installer/src/main/resources/schema/installer-conf.xsd
rename to asterix-events/src/main/resources/schema/installer-conf.xsd
diff --git a/asterix-events/src/main/resources/scripts/execute.sh b/asterix-events/src/main/resources/scripts/execute.sh
old mode 100755
new mode 100644
diff --git a/asterix-events/src/main/resources/scripts/prepare.sh b/asterix-events/src/main/resources/scripts/prepare.sh
old mode 100755
new mode 100644
diff --git a/asterix-external-data/pom.xml b/asterix-external-data/pom.xml
index f8d5ea2..8f58b0f 100644
--- a/asterix-external-data/pom.xml
+++ b/asterix-external-data/pom.xml
@@ -1,18 +1,14 @@
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<!-- ! Copyright 2009-2013 by The Regents of the University of California
+ ! Licensed under the Apache License, Version 2.0 (the "License"); ! you may
+ not use this file except in compliance with the License. ! you may obtain
+ a copy of the License from ! ! http://www.apache.org/licenses/LICENSE-2.0
+ ! ! Unless required by applicable law or agreed to in writing, software !
+ distributed under the License is distributed on an "AS IS" BASIS, ! WITHOUT
+ WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. ! See the
+ License for the specific language governing permissions and ! limitations
+ under the License. ! -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<artifactId>asterix</artifactId>
@@ -34,53 +30,62 @@
</configuration>
</plugin>
<plugin>
- <groupId>org.codehaus.mojo</groupId>
- <artifactId>appassembler-maven-plugin</artifactId>
- <version>1.0</version>
+ <groupId>org.jvnet.jaxb2.maven2</groupId>
+ <artifactId>maven-jaxb2-plugin</artifactId>
<executions>
<execution>
- <configuration>
- <programs>
- <program>
- <mainClass>edu.uci.ics.asterix.drivers.AsterixWebServer</mainClass>
- <name>asterix-web</name>
- </program>
- <program>
- <mainClass>edu.uci.ics.asterix.drivers.AsterixClientDriver</mainClass>
- <name>asterix-cmd</name>
- </program>
- </programs>
- <repositoryLayout>flat</repositoryLayout>
- <repositoryName>lib</repositoryName>
- </configuration>
- <phase>package</phase>
+ <id>configuration</id>
<goals>
- <goal>assemble</goal>
+ <goal>generate</goal>
</goals>
+ <configuration>
+ <schemaDirectory>src/main/resources/schema</schemaDirectory>
+ <schemaIncludes>
+ <include>library.xsd</include>
+ </schemaIncludes>
+ <generatePackage>edu.uci.ics.asterix.external.library</generatePackage>
+ <generateDirectory>${project.build.directory}/generated-sources/configuration</generateDirectory>
+ </configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-surefire-plugin</artifactId>
- <version>2.7.2</version>
+ <artifactId>maven-jar-plugin</artifactId>
+ <version>2.2</version>
<configuration>
- <!-- doesn't work from m2eclipse, currently <additionalClasspathElements>
- <additionalClasspathElement>${basedir}/src/main/resources</additionalClasspathElement>
- </additionalClasspathElements> -->
- <forkMode>pertest</forkMode>
- <argLine>-enableassertions -Xmx${test.heap.size}m
- -Dfile.encoding=UTF-8
- -Djava.util.logging.config.file=src/test/resources/logging.properties</argLine>
<includes>
- <include>**/*TestSuite.java</include>
- <include>**/*Test.java</include>
+ <include>**/*.class</include>
+ <include>**/*.txt</include>
</includes>
</configuration>
+ <executions>
+ <execution>
+ <goals>
+ <goal>test-jar</goal>
+ </goals>
+ <phase>package</phase>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <artifactId>maven-assembly-plugin</artifactId>
+ <version>2.2-beta-5</version>
+ <executions>
+ <execution>
+ <configuration>
+ <descriptor>src/main/assembly/binary-assembly-libzip.xml</descriptor>
+ <finalName>testlib-zip</finalName>
+ </configuration>
+ <phase>package</phase>
+ <goals>
+ <goal>attached</goal>
+ </goals>
+ </execution>
+ </executions>
</plugin>
</plugins>
</build>
-
<dependencies>
<dependency>
<groupId>javax.servlet</groupId>
@@ -108,6 +113,17 @@
<scope>compile</scope>
</dependency>
<dependency>
+ <groupId>edu.uci.ics.asterix</groupId>
+ <artifactId>asterix-metadata</artifactId>
+ <version>0.8.1-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>algebricks-compiler</artifactId>
+ </dependency>
+ <dependency>
<groupId>com.kenai.nbpwr</groupId>
<artifactId>org-apache-commons-io</artifactId>
<version>1.3.1-201002241208</version>
@@ -154,5 +170,4 @@
<version>1.0</version>
</dependency>
</dependencies>
-
-</project>
+</project>
diff --git a/asterix-external-data/src/main/assembly/binary-assembly-libjar.xml b/asterix-external-data/src/main/assembly/binary-assembly-libjar.xml
new file mode 100644
index 0000000..d76174a
--- /dev/null
+++ b/asterix-external-data/src/main/assembly/binary-assembly-libjar.xml
@@ -0,0 +1,19 @@
+<assembly>
+ <id>binary-assembly</id>
+ <formats>
+ <format>jar</format>
+ </formats>
+ <includeBaseDirectory>false</includeBaseDirectory>
+ <fileSets>
+ <fileSet>
+ <directory>target/test-classes</directory>
+ <outputDirectory></outputDirectory>
+ <includes>
+ <include>**</include>
+ </includes>
+ <excludes>
+ <exclude>**.xml</exclude>
+ </excludes>
+ </fileSet>
+ </fileSets>
+</assembly>
diff --git a/asterix-external-data/src/main/assembly/binary-assembly-libzip.xml b/asterix-external-data/src/main/assembly/binary-assembly-libzip.xml
new file mode 100644
index 0000000..bec6e32
--- /dev/null
+++ b/asterix-external-data/src/main/assembly/binary-assembly-libzip.xml
@@ -0,0 +1,23 @@
+<assembly>
+ <id>binary-assembly</id>
+ <formats>
+ <format>zip</format>
+ </formats>
+ <includeBaseDirectory>false</includeBaseDirectory>
+ <fileSets>
+ <fileSet>
+ <directory>target</directory>
+ <outputDirectory></outputDirectory>
+ <includes>
+ <include>*test*.jar</include>
+ </includes>
+ </fileSet>
+ <fileSet>
+ <directory>src/test/resources</directory>
+ <outputDirectory></outputDirectory>
+ <includes>
+ <include>*.xml</include>
+ </includes>
+ </fileSet>
+ </fileSets>
+</assembly>
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/CNNFeedAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/CNNFeedAdapterFactory.java
index f1a3d6c..6880308 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/CNNFeedAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/CNNFeedAdapterFactory.java
@@ -14,21 +14,74 @@
*/
package edu.uci.ics.asterix.external.adapter.factory;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
import java.util.Map;
import edu.uci.ics.asterix.external.dataset.adapter.CNNFeedAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
/**
- * A factory class for creating the @see {CNNFeedAdapter}.
+ * A factory class for creating the @see {CNNFeedAdapter}.
*/
-public class CNNFeedAdapterFactory implements ITypedDatasetAdapterFactory {
+public class CNNFeedAdapterFactory implements ITypedAdapterFactory {
private static final long serialVersionUID = 1L;
+ private Map<String, String> configuration;
+
+ private List<String> feedURLs = new ArrayList<String>();
+ private static Map<String, String> topicFeeds = new HashMap<String, String>();
+ private ARecordType recordType;
+
+ public static final String KEY_RSS_URL = "topic";
+ public static final String KEY_INTERVAL = "interval";
+ public static final String TOP_STORIES = "topstories";
+ public static final String WORLD = "world";
+ public static final String US = "us";
+ public static final String SPORTS = "sports";
+ public static final String BUSINESS = "business";
+ public static final String POLITICS = "politics";
+ public static final String CRIME = "crime";
+ public static final String TECHNOLOGY = "technology";
+ public static final String HEALTH = "health";
+ public static final String ENTERNTAINMENT = "entertainemnt";
+ public static final String TRAVEL = "travel";
+ public static final String LIVING = "living";
+ public static final String VIDEO = "video";
+ public static final String STUDENT = "student";
+ public static final String POPULAR = "popular";
+ public static final String RECENT = "recent";
+
+ private void initTopics() {
+ topicFeeds.put(TOP_STORIES, "http://rss.cnn.com/rss/cnn_topstories.rss");
+ topicFeeds.put(WORLD, "http://rss.cnn.com/rss/cnn_world.rss");
+ topicFeeds.put(US, "http://rss.cnn.com/rss/cnn_us.rss");
+ topicFeeds.put(SPORTS, "http://rss.cnn.com/rss/si_topstories.rss");
+ topicFeeds.put(BUSINESS, "http://rss.cnn.com/rss/money_latest.rss");
+ topicFeeds.put(POLITICS, "http://rss.cnn.com/rss/cnn_allpolitics.rss");
+ topicFeeds.put(CRIME, "http://rss.cnn.com/rss/cnn_crime.rss");
+ topicFeeds.put(TECHNOLOGY, "http://rss.cnn.com/rss/cnn_tech.rss");
+ topicFeeds.put(HEALTH, "http://rss.cnn.com/rss/cnn_health.rss");
+ topicFeeds.put(ENTERNTAINMENT, "http://rss.cnn.com/rss/cnn_showbiz.rss");
+ topicFeeds.put(LIVING, "http://rss.cnn.com/rss/cnn_living.rss");
+ topicFeeds.put(VIDEO, "http://rss.cnn.com/rss/cnn_freevideo.rss");
+ topicFeeds.put(TRAVEL, "http://rss.cnn.com/rss/cnn_travel.rss");
+ topicFeeds.put(STUDENT, "http://rss.cnn.com/rss/cnn_studentnews.rss");
+ topicFeeds.put(POPULAR, "http://rss.cnn.com/rss/cnn_mostpopular.rss");
+ topicFeeds.put(RECENT, "http://rss.cnn.com/rss/cnn_latest.rss");
+ }
+
@Override
- public IDatasourceAdapter createAdapter(Map<String, Object> configuration) throws Exception {
- CNNFeedAdapter cnnFeedAdapter = new CNNFeedAdapter();
- cnnFeedAdapter.configure(configuration);
+ public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
+ CNNFeedAdapter cnnFeedAdapter = new CNNFeedAdapter(configuration, recordType, ctx);
return cnnFeedAdapter;
}
@@ -37,4 +90,61 @@
return "cnn_feed";
}
+ @Override
+ public AdapterType getAdapterType() {
+ return AdapterType.TYPED;
+ }
+
+ @Override
+ public void configure(Map<String, String> configuration) throws Exception {
+ this.configuration = configuration;
+ String rssURLProperty = (String) configuration.get(KEY_RSS_URL);
+ if (rssURLProperty == null) {
+ throw new IllegalArgumentException("no rss url provided");
+ }
+ initializeFeedURLs(rssURLProperty);
+ recordType = new ARecordType("FeedRecordType", new String[] { "id", "title", "description", "link" },
+ new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING },
+ false);
+
+ }
+
+ private void initializeFeedURLs(String rssURLProperty) {
+ feedURLs.clear();
+ String[] rssTopics = rssURLProperty.split(",");
+ initTopics();
+ for (String topic : rssTopics) {
+ String feedURL = topicFeeds.get(topic);
+ if (feedURL == null) {
+ throw new IllegalArgumentException(" unknown topic :" + topic + " please choose from the following "
+ + getValidTopics());
+ }
+ feedURLs.add(feedURL);
+ }
+ }
+
+ private static String getValidTopics() {
+ StringBuilder builder = new StringBuilder();
+ for (String key : topicFeeds.keySet()) {
+ builder.append(key);
+ builder.append(" ");
+ }
+ return new String(builder);
+ }
+
+ @Override
+ public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+ return new AlgebricksCountPartitionConstraint(feedURLs.size());
+ }
+
+ @Override
+ public SupportedOperation getSupportedOperations() {
+ return SupportedOperation.READ;
+ }
+
+ @Override
+ public ARecordType getAdapterOutputType() {
+ return recordType;
+ }
+
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java
index 4ca3d72..af057c9 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java
@@ -14,6 +14,7 @@
*/
package edu.uci.ics.asterix.external.adapter.factory;
+import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
@@ -22,10 +23,18 @@
import org.apache.hadoop.mapred.JobConf;
import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
+import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.ICCContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.hdfs.dataflow.ConfFactory;
import edu.uci.ics.hyracks.hdfs.dataflow.InputSplitsFactory;
import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
@@ -34,7 +43,7 @@
* A factory class for creating an instance of HDFSAdapter
*/
@SuppressWarnings("deprecation")
-public class HDFSAdapterFactory implements IGenericDatasetAdapterFactory {
+public class HDFSAdapterFactory extends StreamBasedAdapterFactory implements IGenericAdapterFactory {
private static final long serialVersionUID = 1L;
public static final String HDFS_ADAPTER_NAME = "hdfs";
@@ -52,7 +61,22 @@
private boolean executed[];
private InputSplitsFactory inputSplitsFactory;
private ConfFactory confFactory;
- private boolean setup = false;
+ private IAType atype;
+ private boolean configured = false;
+ public static Scheduler hdfsScheduler;
+ private static boolean initialized = false;
+
+ private static Scheduler initializeHDFSScheduler() {
+ ICCContext ccContext = AsterixAppContextInfo.getInstance().getCCApplicationContext().getCCContext();
+ Scheduler scheduler = null;
+ try {
+ scheduler = new Scheduler(ccContext.getClusterControllerInfo().getClientNetAddress(), ccContext
+ .getClusterControllerInfo().getClientNetPort());
+ } catch (HyracksException e) {
+ throw new IllegalStateException("Cannot obtain hdfs scheduler");
+ }
+ return scheduler;
+ }
private static final Map<String, String> formatClassNames = initInputFormatMap();
@@ -64,30 +88,12 @@
}
@Override
- public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
- if (!setup) {
- /** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
- configureJobConf(configuration);
- JobConf conf = configureJobConf(configuration);
- confFactory = new ConfFactory(conf);
-
- clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
- int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
-
- InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
- inputSplitsFactory = new InputSplitsFactory(inputSplits);
-
- Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
- readSchedule = scheduler.getLocationConstraints(inputSplits);
- executed = new boolean[readSchedule.length];
- Arrays.fill(executed, false);
-
- setup = true;
- }
+ public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
JobConf conf = confFactory.getConf();
InputSplit[] inputSplits = inputSplitsFactory.getSplits();
- HDFSAdapter hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
- hdfsAdapter.configure(configuration);
+ String nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+ HDFSAdapter hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, nodeName,
+ parserFactory, ctx);
return hdfsAdapter;
}
@@ -96,7 +102,7 @@
return HDFS_ADAPTER_NAME;
}
- private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
+ private JobConf configureJobConf(Map<String, String> configuration) throws Exception {
JobConf conf = new JobConf();
conf.set("fs.default.name", ((String) configuration.get(KEY_HDFS_URL)).trim());
conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
@@ -107,4 +113,64 @@
return conf;
}
+ @Override
+ public AdapterType getAdapterType() {
+ return AdapterType.GENERIC;
+ }
+
+ @Override
+ public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+ if (!configured) {
+ throw new IllegalStateException("Adapter factory has not been configured yet");
+ }
+ return (AlgebricksPartitionConstraint) clusterLocations;
+ }
+
+ @Override
+ public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+ if (!initialized) {
+ hdfsScheduler = initializeHDFSScheduler();
+ initialized = true;
+ }
+ this.configuration = configuration;
+ JobConf conf = configureJobConf(configuration);
+ confFactory = new ConfFactory(conf);
+
+ clusterLocations = getClusterLocations();
+ int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
+
+ InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+ inputSplitsFactory = new InputSplitsFactory(inputSplits);
+
+ readSchedule = hdfsScheduler.getLocationConstraints(inputSplits);
+ executed = new boolean[readSchedule.length];
+ Arrays.fill(executed, false);
+ configured = true;
+
+ atype = (IAType) outputType;
+ configureFormat(atype);
+ }
+
+ @Override
+ public SupportedOperation getSupportedOperations() {
+ return SupportedOperation.READ;
+ }
+
+ private static AlgebricksPartitionConstraint getClusterLocations() {
+ ArrayList<String> locs = new ArrayList<String>();
+ Map<String, String[]> stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
+ for (String i : stores.keySet()) {
+ String[] nodeStores = stores.get(i);
+ int numIODevices = AsterixClusterProperties.INSTANCE.getNumberOfIODevices(i);
+ for (int j = 0; j < nodeStores.length; j++) {
+ for (int k = 0; k < numIODevices; k++) {
+ locs.add(i);
+ }
+ }
+ }
+ String[] cluster = new String[locs.size()];
+ cluster = locs.toArray(cluster);
+ return new AlgebricksAbsolutePartitionConstraint(cluster);
+ }
+
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java
index 409eb7a..991dadb 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java
@@ -23,10 +23,14 @@
import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.HiveAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.hdfs.dataflow.ConfFactory;
import edu.uci.ics.hyracks.hdfs.dataflow.InputSplitsFactory;
import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
@@ -35,7 +39,7 @@
* A factory class for creating an instance of HiveAdapter
*/
@SuppressWarnings("deprecation")
-public class HiveAdapterFactory implements IGenericDatasetAdapterFactory {
+public class HiveAdapterFactory extends StreamBasedAdapterFactory implements IGenericAdapterFactory {
private static final long serialVersionUID = 1L;
public static final String HDFS_ADAPTER_NAME = "hdfs";
@@ -65,7 +69,8 @@
private InputSplitsFactory inputSplitsFactory;
private ConfFactory confFactory;
private transient AlgebricksPartitionConstraint clusterLocations;
- private boolean setup = false;
+ private boolean configured = false;
+ private IAType atype;
private static final Map<String, String> formatClassNames = initInputFormatMap();
@@ -77,30 +82,12 @@
}
@Override
- public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
- if (!setup) {
- /** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
- configureJobConf(configuration);
- JobConf conf = configureJobConf(configuration);
- confFactory = new ConfFactory(conf);
-
- clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
- int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
-
- InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
- inputSplitsFactory = new InputSplitsFactory(inputSplits);
-
- Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
- readSchedule = scheduler.getLocationConstraints(inputSplits);
- executed = new boolean[readSchedule.length];
- Arrays.fill(executed, false);
-
- setup = true;
- }
+ public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
JobConf conf = confFactory.getConf();
InputSplit[] inputSplits = inputSplitsFactory.getSplits();
- HiveAdapter hiveAdapter = new HiveAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
- hiveAdapter.configure(configuration);
+ String nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+ HiveAdapter hiveAdapter = new HiveAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations,
+ nodeName, parserFactory, ctx);
return hiveAdapter;
}
@@ -109,7 +96,43 @@
return "hive";
}
- private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
+ @Override
+ public AdapterType getAdapterType() {
+ return AdapterType.GENERIC;
+ }
+
+ @Override
+ public SupportedOperation getSupportedOperations() {
+ return SupportedOperation.READ;
+ }
+
+ @Override
+ public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+ if (!configured) {
+ /** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
+ configureJobConf(configuration);
+ JobConf conf = configureJobConf(configuration);
+ confFactory = new ConfFactory(conf);
+
+ clusterLocations = AsterixClusterProperties.INSTANCE.getClusterLocations();
+ int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
+
+ InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+ inputSplitsFactory = new InputSplitsFactory(inputSplits);
+
+ Scheduler scheduler = HDFSAdapterFactory.hdfsScheduler;
+ readSchedule = scheduler.getLocationConstraints(inputSplits);
+ executed = new boolean[readSchedule.length];
+ Arrays.fill(executed, false);
+
+ atype = (IAType) outputType;
+ configureFormat(atype);
+ configured = true;
+ }
+
+ }
+
+ private JobConf configureJobConf(Map<String, String> configuration) throws Exception {
JobConf conf = new JobConf();
/** configure hive */
@@ -121,7 +144,7 @@
tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + tablePath + ".db" + "/"
+ configuration.get(HIVE_TABLE);
}
- configuration.put(HDFSAdapter.KEY_PATH, tablePath);
+ configuration.put(HDFSAdapterFactory.KEY_PATH, tablePath);
if (!configuration.get(KEY_FORMAT).equals(FORMAT_DELIMITED_TEXT)) {
throw new IllegalArgumentException("format" + configuration.get(KEY_FORMAT) + " is not supported");
}
@@ -141,4 +164,11 @@
(String) formatClassNames.get(((String) configuration.get(KEY_INPUT_FORMAT)).trim()));
return conf;
}
+
+ @Override
+ public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IAdapterFactory.java
deleted file mode 100644
index 5cb6777..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IAdapterFactory.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.external.adapter.factory;
-
-import java.io.Serializable;
-
-/**
- * Base interface for IGenericDatasetAdapterFactory and ITypedDatasetAdapterFactory.
- * Acts as a marker interface indicating that the implementation provides functionality
- * for creating an adapter.
- */
-public interface IAdapterFactory extends Serializable {
-
- /**
- * Returns the display name corresponding to the Adapter type that is created by the factory.
- *
- * @return the display name
- */
- public String getName();
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java
deleted file mode 100644
index 0a178a7..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.external.adapter.factory;
-
-import java.util.Map;
-
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
-import edu.uci.ics.asterix.om.types.IAType;
-
-/**
- * A base interface for an adapter factory that creates instance of an adapter kind that
- * is 'generic' in nature. A 'typed' adapter returns records with a configurable datatype.
- */
-public interface IGenericDatasetAdapterFactory extends IAdapterFactory {
-
- public static final String KEY_TYPE_NAME = "output-type-name";
-
- /**
- * Creates an instance of IDatasourceAdapter.
- *
- * @param configuration
- * The configuration parameters for the adapter that is instantiated.
- * The passed-in configuration is used to configure the created instance of the adapter.
- * @param atype
- * The type for the ADM records that are returned by the adapter.
- * @return An instance of IDatasourceAdapter.
- * @throws Exception
- */
- public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception;
-
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/ITypedDatasetAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/ITypedDatasetAdapterFactory.java
deleted file mode 100644
index 674bf52..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/ITypedDatasetAdapterFactory.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.external.adapter.factory;
-
-import java.util.Map;
-
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
-
-/**
- * A base interface for an adapter factory that creates instance of an adapter kind that
- * is 'typed' in nature. A 'typed' adapter returns records with a pre-defined datatype.
- */
-public interface ITypedDatasetAdapterFactory extends IAdapterFactory {
-
- /**
- * Creates an instance of IDatasourceAdapter.
- *
- * @param configuration
- * The configuration parameters for the adapter that is instantiated.
- * The passed-in configuration is used to configure the created instance of the adapter.
- * @return An instance of IDatasourceAdapter.
- * @throws Exception
- */
- public IDatasourceAdapter createAdapter(Map<String, Object> configuration) throws Exception;
-
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
index e680232..7c7b2a3 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
@@ -14,25 +14,43 @@
*/
package edu.uci.ics.asterix.external.adapter.factory;
+import java.io.File;
import java.util.Map;
+import java.util.logging.Level;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter;
+import edu.uci.ics.asterix.external.util.DNSResolverFactory;
+import edu.uci.ics.asterix.external.util.INodeResolver;
+import edu.uci.ics.asterix.external.util.INodeResolverFactory;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
/**
* Factory class for creating an instance of NCFileSystemAdapter. An
* NCFileSystemAdapter reads external data residing on the local file system of
* an NC.
*/
-public class NCFileSystemAdapterFactory implements IGenericDatasetAdapterFactory {
+public class NCFileSystemAdapterFactory extends StreamBasedAdapterFactory implements IGenericAdapterFactory {
private static final long serialVersionUID = 1L;
+
public static final String NC_FILE_SYSTEM_ADAPTER_NAME = "localfs";
+ private static final INodeResolver DEFAULT_NODE_RESOLVER = new DNSResolverFactory().createNodeResolver();
+
+ private IAType sourceDatatype;
+ private FileSplit[] fileSplits;
+
@Override
- public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
- NCFileSystemAdapter fsAdapter = new NCFileSystemAdapter(atype);
- fsAdapter.configure(configuration);
+ public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
+ NCFileSystemAdapter fsAdapter = new NCFileSystemAdapter(fileSplits, parserFactory, sourceDatatype, ctx);
return fsAdapter;
}
@@ -40,4 +58,88 @@
public String getName() {
return NC_FILE_SYSTEM_ADAPTER_NAME;
}
+
+ @Override
+ public AdapterType getAdapterType() {
+ return AdapterType.GENERIC;
+ }
+
+ @Override
+ public SupportedOperation getSupportedOperations() {
+ return SupportedOperation.READ;
+ }
+
+ @Override
+ public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+ this.configuration = configuration;
+ String[] splits = ((String) configuration.get(KEY_PATH)).split(",");
+ IAType sourceDatatype = (IAType) outputType;
+ configureFileSplits(splits);
+ configureFormat(sourceDatatype);
+ }
+
+ @Override
+ public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+ return configurePartitionConstraint();
+ }
+
+ private void configureFileSplits(String[] splits) throws AsterixException {
+ if (fileSplits == null) {
+ fileSplits = new FileSplit[splits.length];
+ String nodeName;
+ String nodeLocalPath;
+ int count = 0;
+ String trimmedValue;
+ for (String splitPath : splits) {
+ trimmedValue = splitPath.trim();
+ if (!trimmedValue.contains("://")) {
+ throw new AsterixException("Invalid path: " + splitPath
+ + "\nUsage- path=\"Host://Absolute File Path\"");
+ }
+ nodeName = trimmedValue.split(":")[0];
+ nodeLocalPath = trimmedValue.split("://")[1];
+ FileSplit fileSplit = new FileSplit(nodeName, new FileReference(new File(nodeLocalPath)));
+ fileSplits[count++] = fileSplit;
+ }
+ }
+ }
+
+ private AlgebricksPartitionConstraint configurePartitionConstraint() throws AsterixException {
+ String[] locs = new String[fileSplits.length];
+ String location;
+ for (int i = 0; i < fileSplits.length; i++) {
+ location = getNodeResolver().resolveNode(fileSplits[i].getNodeName());
+ locs[i] = location;
+ }
+ return new AlgebricksAbsolutePartitionConstraint(locs);
+ }
+
+ protected INodeResolver getNodeResolver() {
+ if (nodeResolver == null) {
+ nodeResolver = initNodeResolver();
+ }
+ return nodeResolver;
+ }
+
+ private static INodeResolver initNodeResolver() {
+ INodeResolver nodeResolver = null;
+ String configuredNodeResolverFactory = System.getProperty(NODE_RESOLVER_FACTORY_PROPERTY);
+ if (configuredNodeResolverFactory != null) {
+ try {
+ nodeResolver = ((INodeResolverFactory) (Class.forName(configuredNodeResolverFactory).newInstance()))
+ .createNodeResolver();
+
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.log(Level.WARNING, "Unable to create node resolver from the configured classname "
+ + configuredNodeResolverFactory + "\n" + e.getMessage());
+ }
+ nodeResolver = DEFAULT_NODE_RESOLVER;
+ }
+ } else {
+ nodeResolver = DEFAULT_NODE_RESOLVER;
+ }
+ return nodeResolver;
+ }
+
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PullBasedTwitterAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PullBasedTwitterAdapterFactory.java
index 232d5e8..9c137e1 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PullBasedTwitterAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PullBasedTwitterAdapterFactory.java
@@ -16,23 +16,31 @@
import java.util.Map;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.PullBasedTwitterAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
/**
* Factory class for creating an instance of PullBasedTwitterAdapter.
* This adapter provides the functionality of fetching tweets from Twitter service
* via pull-based Twitter API.
*/
-public class PullBasedTwitterAdapterFactory implements ITypedDatasetAdapterFactory {
+public class PullBasedTwitterAdapterFactory implements ITypedAdapterFactory {
private static final long serialVersionUID = 1L;
public static final String PULL_BASED_TWITTER_ADAPTER_NAME = "pull_twitter";
+ private Map<String, String> configuration;
+ private static ARecordType recordType;
+
@Override
- public IDatasourceAdapter createAdapter(Map<String, Object> configuration) throws Exception {
- PullBasedTwitterAdapter twitterAdapter = new PullBasedTwitterAdapter();
- twitterAdapter.configure(configuration);
- return twitterAdapter;
+ public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
+ return new PullBasedTwitterAdapter(configuration, ctx);
}
@Override
@@ -40,4 +48,39 @@
return PULL_BASED_TWITTER_ADAPTER_NAME;
}
+ @Override
+ public AdapterType getAdapterType() {
+ return AdapterType.TYPED;
+ }
+
+ @Override
+ public SupportedOperation getSupportedOperations() {
+ return SupportedOperation.READ;
+ }
+
+ @Override
+ public void configure(Map<String, String> configuration) throws Exception {
+ this.configuration = configuration;
+ if (recordType != null) {
+ String[] fieldNames = { "id", "username", "location", "text", "timestamp" };
+ IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
+ BuiltinType.ASTRING };
+ try {
+ recordType = new ARecordType("FeedRecordType", fieldNames, fieldTypes, false);
+ } catch (Exception e) {
+ throw new IllegalStateException("Unable to create adapter output type");
+ }
+ }
+ }
+
+ @Override
+ public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+ return new AlgebricksCountPartitionConstraint(1);
+ }
+
+ @Override
+ public ARecordType getAdapterOutputType() {
+ return recordType;
+ }
+
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/RSSFeedAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/RSSFeedAdapterFactory.java
index ab18455..cc366f6 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/RSSFeedAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/RSSFeedAdapterFactory.java
@@ -14,23 +14,39 @@
*/
package edu.uci.ics.asterix.external.adapter.factory;
+import java.util.ArrayList;
+import java.util.List;
import java.util.Map;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.RSSFeedAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
/**
* Factory class for creating an instance of @see {RSSFeedAdapter}.
* RSSFeedAdapter provides the functionality of fetching an RSS based feed.
*/
-public class RSSFeedAdapterFactory implements ITypedDatasetAdapterFactory {
+public class RSSFeedAdapterFactory implements ITypedAdapterFactory {
private static final long serialVersionUID = 1L;
public static final String RSS_FEED_ADAPTER_NAME = "rss_feed";
+ public static final String KEY_RSS_URL = "url";
+ public static final String KEY_INTERVAL = "interval";
+
+ private Map<String, String> configuration;
+ private ARecordType recordType;
+ private List<String> feedURLs = new ArrayList<String>();
+
@Override
- public IDatasourceAdapter createAdapter(Map<String, Object> configuration) throws Exception {
- RSSFeedAdapter rssFeedAdapter = new RSSFeedAdapter();
- rssFeedAdapter.configure(configuration);
+ public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
+ RSSFeedAdapter rssFeedAdapter = new RSSFeedAdapter(configuration, recordType, ctx);
return rssFeedAdapter;
}
@@ -39,4 +55,51 @@
return "rss_feed";
}
+ @Override
+ public AdapterType getAdapterType() {
+ return AdapterType.TYPED;
+ }
+
+ @Override
+ public SupportedOperation getSupportedOperations() {
+ return SupportedOperation.READ;
+ }
+
+ @Override
+ public void configure(Map<String, String> configuration) throws Exception {
+ this.configuration = configuration;
+ String rssURLProperty = (String) configuration.get(KEY_RSS_URL);
+ if (rssURLProperty == null) {
+ throw new IllegalArgumentException("no rss url provided");
+ }
+ initializeFeedURLs(rssURLProperty);
+ configurePartitionConstraints();
+ recordType = new ARecordType("FeedRecordType", new String[] { "id", "title", "description", "link" },
+ new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING },
+ false);
+
+ }
+
+ @Override
+ public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+ return new AlgebricksCountPartitionConstraint(feedURLs.size());
+ }
+
+ private void initializeFeedURLs(String rssURLProperty) {
+ feedURLs.clear();
+ String[] feedURLProperty = rssURLProperty.split(",");
+ for (String feedURL : feedURLProperty) {
+ feedURLs.add(feedURL);
+ }
+ }
+
+ protected void configurePartitionConstraints() {
+
+ }
+
+ @Override
+ public ARecordType getAdapterOutputType() {
+ return recordType;
+ }
+
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/StreamBasedAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/StreamBasedAdapterFactory.java
new file mode 100644
index 0000000..02ffdda
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/StreamBasedAdapterFactory.java
@@ -0,0 +1,120 @@
+package edu.uci.ics.asterix.external.adapter.factory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.external.util.DNSResolverFactory;
+import edu.uci.ics.asterix.external.util.INodeResolver;
+import edu.uci.ics.asterix.external.util.INodeResolverFactory;
+import edu.uci.ics.asterix.metadata.feeds.ConditionalPushTupleParserFactory;
+import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.runtime.operators.file.AdmSchemafullRecordParserFactory;
+import edu.uci.ics.asterix.runtime.operators.file.NtDelimitedDataTupleParserFactory;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.FloatParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.LongParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+public abstract class StreamBasedAdapterFactory implements IAdapterFactory {
+
+ private static final long serialVersionUID = 1L;
+ protected static final Logger LOGGER = Logger.getLogger(StreamBasedAdapterFactory.class.getName());
+
+ protected Map<String, String> configuration;
+ protected static INodeResolver nodeResolver;
+
+ public static final String KEY_FORMAT = "format";
+ public static final String KEY_PARSER_FACTORY = "parser";
+ public static final String KEY_DELIMITER = "delimiter";
+ public static final String KEY_PATH = "path";
+ public static final String KEY_SOURCE_DATATYPE = "output-type-name";
+ public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
+ public static final String FORMAT_ADM = "adm";
+ public static final String NODE_RESOLVER_FACTORY_PROPERTY = "node.Resolver";
+ public static final String BATCH_SIZE = "batch-size";
+ public static final String BATCH_INTERVAL = "batch-interval";
+
+ protected ITupleParserFactory parserFactory;
+ protected ITupleParser parser;
+
+ protected static final HashMap<ATypeTag, IValueParserFactory> typeToValueParserFactMap = new HashMap<ATypeTag, IValueParserFactory>();
+ static {
+ typeToValueParserFactMap.put(ATypeTag.INT32, IntegerParserFactory.INSTANCE);
+ typeToValueParserFactMap.put(ATypeTag.FLOAT, FloatParserFactory.INSTANCE);
+ typeToValueParserFactMap.put(ATypeTag.DOUBLE, DoubleParserFactory.INSTANCE);
+ typeToValueParserFactMap.put(ATypeTag.INT64, LongParserFactory.INSTANCE);
+ typeToValueParserFactMap.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
+ }
+
+ protected ITupleParserFactory getDelimitedDataTupleParserFactory(ARecordType recordType, boolean conditionalPush)
+ throws AsterixException {
+ int n = recordType.getFieldTypes().length;
+ IValueParserFactory[] fieldParserFactories = new IValueParserFactory[n];
+ for (int i = 0; i < n; i++) {
+ ATypeTag tag = recordType.getFieldTypes()[i].getTypeTag();
+ IValueParserFactory vpf = typeToValueParserFactMap.get(tag);
+ if (vpf == null) {
+ throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
+ }
+ fieldParserFactories[i] = vpf;
+ }
+ String delimiterValue = (String) configuration.get(KEY_DELIMITER);
+ if (delimiterValue != null && delimiterValue.length() > 1) {
+ throw new AsterixException("improper delimiter");
+ }
+
+ Character delimiter = delimiterValue.charAt(0);
+
+ return conditionalPush ? new ConditionalPushTupleParserFactory(recordType, fieldParserFactories, delimiter,
+ configuration) : new NtDelimitedDataTupleParserFactory(recordType, fieldParserFactories, delimiter);
+ }
+
+ protected ITupleParserFactory getADMDataTupleParserFactory(ARecordType recordType, boolean conditionalPush)
+ throws AsterixException {
+ try {
+ return conditionalPush ? new ConditionalPushTupleParserFactory(recordType, configuration)
+ : new AdmSchemafullRecordParserFactory(recordType);
+ } catch (Exception e) {
+ throw new AsterixException(e);
+ }
+
+ }
+
+ protected void configureFormat(IAType sourceDatatype) throws Exception {
+ String propValue = (String) configuration.get(BATCH_SIZE);
+ int batchSize = propValue != null ? Integer.parseInt(propValue) : -1;
+ propValue = (String) configuration.get(BATCH_INTERVAL);
+ long batchInterval = propValue != null ? Long.parseLong(propValue) : -1;
+ boolean conditionalPush = batchSize > 0 || batchInterval > 0;
+
+ String parserFactoryClassname = (String) configuration.get(KEY_PARSER_FACTORY);
+ if (parserFactoryClassname == null) {
+ String specifiedFormat = (String) configuration.get(KEY_FORMAT);
+ if (specifiedFormat == null) {
+ throw new IllegalArgumentException(" Unspecified data format");
+ } else if (FORMAT_DELIMITED_TEXT.equalsIgnoreCase(specifiedFormat)) {
+ parserFactory = getDelimitedDataTupleParserFactory((ARecordType) sourceDatatype, conditionalPush);
+ } else if (FORMAT_ADM.equalsIgnoreCase((String) configuration.get(KEY_FORMAT))) {
+ parserFactory = getADMDataTupleParserFactory((ARecordType) sourceDatatype, conditionalPush);
+ } else {
+ throw new IllegalArgumentException(" format " + configuration.get(KEY_FORMAT) + " not supported");
+ }
+ } else {
+ parserFactory = (ITupleParserFactory) Class.forName(parserFactoryClassname).newInstance();
+ }
+
+ }
+
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorDescriptor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorDescriptor.java
deleted file mode 100644
index a48cfb8..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorDescriptor.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.external.data.operator;
-
-import java.util.Map;
-
-import edu.uci.ics.asterix.external.adapter.factory.IAdapterFactory;
-import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
-import edu.uci.ics.asterix.external.adapter.factory.ITypedDatasetAdapterFactory;
-import edu.uci.ics.asterix.external.dataset.adapter.ITypedDatasourceAdapter;
-import edu.uci.ics.asterix.external.feed.lifecycle.FeedId;
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-
-/**
- * Operator responsible for ingesting data from an external source. This
- * operator uses a (configurable) adapter associated with the feed dataset.
- */
-public class FeedIntakeOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
-
- private static final long serialVersionUID = 1L;
-
- private final String adapterFactoryClassName;
- private final Map<String, Object> adapterConfiguration;
- private final IAType atype;
- private final FeedId feedId;
- private final IAdapterFactory datasourceAdapterFactory;
-
- public FeedIntakeOperatorDescriptor(JobSpecification spec, FeedId feedId, String adapter,
- Map<String, Object> arguments, ARecordType atype, RecordDescriptor rDesc,
- IAdapterFactory datasourceAdapterFactory) {
- super(spec, 1, 1);
- recordDescriptors[0] = rDesc;
- this.adapterFactoryClassName = adapter;
- this.adapterConfiguration = arguments;
- this.atype = atype;
- this.feedId = feedId;
- this.datasourceAdapterFactory = datasourceAdapterFactory;
- }
-
- public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
- throws HyracksDataException {
- ITypedDatasourceAdapter adapter;
- try {
- if (datasourceAdapterFactory instanceof IGenericDatasetAdapterFactory) {
- adapter = (ITypedDatasourceAdapter) ((IGenericDatasetAdapterFactory) datasourceAdapterFactory)
- .createAdapter(adapterConfiguration, atype);
- } else if (datasourceAdapterFactory instanceof ITypedDatasetAdapterFactory) {
- adapter = (ITypedDatasourceAdapter) ((ITypedDatasetAdapterFactory) datasourceAdapterFactory)
- .createAdapter(adapterConfiguration);
- } else {
- throw new IllegalStateException(" Unknown adapter factory type for " + adapterFactoryClassName);
- }
- adapter.initialize(ctx);
- } catch (Exception e) {
- throw new HyracksDataException("initialization of adapter failed", e);
- }
- return new FeedIntakeOperatorNodePushable(feedId, adapter, partition);
- }
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorNodePushable.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorNodePushable.java
deleted file mode 100644
index 31470f3..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorNodePushable.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.external.data.operator;
-
-import java.nio.ByteBuffer;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import edu.uci.ics.asterix.common.api.AsterixThreadExecutor;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
-import edu.uci.ics.asterix.external.feed.lifecycle.AlterFeedMessage;
-import edu.uci.ics.asterix.external.feed.lifecycle.FeedId;
-import edu.uci.ics.asterix.external.feed.lifecycle.FeedManager;
-import edu.uci.ics.asterix.external.feed.lifecycle.IFeedManager;
-import edu.uci.ics.asterix.external.feed.lifecycle.IFeedMessage;
-import edu.uci.ics.asterix.feed.managed.adapter.IManagedFeedAdapter;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-
-/**
- * The runtime for @see{FeedIntakeOperationDescriptor}
- */
-public class FeedIntakeOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
-
- private final IDatasourceAdapter adapter;
- private final int partition;
- private final IFeedManager feedManager;
- private final FeedId feedId;
- private final LinkedBlockingQueue<IFeedMessage> inbox;
- private FeedInboxMonitor feedInboxMonitor;
-
- public FeedIntakeOperatorNodePushable(FeedId feedId, IDatasourceAdapter adapter, int partition) {
- this.adapter = adapter;
- this.partition = partition;
- this.feedManager = (IFeedManager) FeedManager.INSTANCE;
- this.feedId = feedId;
- inbox = new LinkedBlockingQueue<IFeedMessage>();
- }
-
- @Override
- public void open() throws HyracksDataException {
- if (adapter instanceof IManagedFeedAdapter) {
- feedInboxMonitor = new FeedInboxMonitor((IManagedFeedAdapter) adapter, inbox, partition);
- AsterixThreadExecutor.INSTANCE.execute(feedInboxMonitor);
- feedManager.registerFeedMsgQueue(feedId, inbox);
- }
- writer.open();
- try {
- adapter.start(partition, writer);
- } catch (Exception e) {
- e.printStackTrace();
- throw new HyracksDataException(e);
- /*
- we do not throw an exception, but allow the operator to close
- gracefully throwing an exception here would result in a job abort and a
- transaction roll back that undoes all the work done so far.
- */
-
- } finally {
- writer.close();
- if (adapter instanceof IManagedFeedAdapter) {
- feedManager.unregisterFeedMsgQueue(feedId, inbox);
- }
- }
- }
-
- @Override
- public void fail() throws HyracksDataException {
- writer.close();
- }
-
- @Override
- public void close() throws HyracksDataException {
-
- }
-
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- // do nothing
- }
-}
-
-class FeedInboxMonitor extends Thread {
-
- private LinkedBlockingQueue<IFeedMessage> inbox;
- private final IManagedFeedAdapter adapter;
-
- public FeedInboxMonitor(IManagedFeedAdapter adapter, LinkedBlockingQueue<IFeedMessage> inbox, int partition) {
- this.inbox = inbox;
- this.adapter = adapter;
- }
-
- @Override
- public void run() {
- while (true) {
- try {
- IFeedMessage feedMessage = inbox.take();
- switch (feedMessage.getMessageType()) {
- case STOP:
- adapter.stop();
- break;
- case ALTER:
- adapter.alter(((AlterFeedMessage) feedMessage).getAlteredConfParams());
- break;
- }
- } catch (InterruptedException ie) {
- break;
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
- }
-
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedMessageOperatorNodePushable.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedMessageOperatorNodePushable.java
deleted file mode 100644
index b6bacef..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedMessageOperatorNodePushable.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.external.data.operator;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import edu.uci.ics.asterix.external.feed.lifecycle.FeedId;
-import edu.uci.ics.asterix.external.feed.lifecycle.FeedManager;
-import edu.uci.ics.asterix.external.feed.lifecycle.IFeedManager;
-import edu.uci.ics.asterix.external.feed.lifecycle.IFeedMessage;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
-
-/**
- * Runtime for the @see{FeedMessageOperatorDescriptor}
- */
-public class FeedMessageOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
-
- private final FeedId feedId;
- private final List<IFeedMessage> feedMessages;
- private IFeedManager feedManager;
-
- public FeedMessageOperatorNodePushable(IHyracksTaskContext ctx, FeedId feedId, List<IFeedMessage> feedMessages,
- boolean applyToAll, int partition, int nPartitions) {
- this.feedId = feedId;
- if (applyToAll) {
- this.feedMessages = feedMessages;
- } else {
- this.feedMessages = new ArrayList<IFeedMessage>();
- feedMessages.add(feedMessages.get(partition));
- }
- feedManager = (IFeedManager) FeedManager.INSTANCE;
- }
-
- @Override
- public void initialize() throws HyracksDataException {
- try {
- writer.open();
- for (IFeedMessage feedMessage : feedMessages) {
- feedManager.deliverMessage(feedId, feedMessage);
- }
- } catch (Exception e) {
- throw new HyracksDataException(e);
- } finally {
- writer.close();
- }
- }
-
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/AbstractDatasourceAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/AbstractDatasourceAdapter.java
deleted file mode 100644
index f96a030..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/AbstractDatasourceAdapter.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.external.dataset.adapter;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.FloatParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.LongParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-
-/**
- * Represents the base class that is required to be extended by every
- * implementation of the IDatasourceAdapter interface.
- */
-public abstract class AbstractDatasourceAdapter implements IDatasourceAdapter {
-
- private static final long serialVersionUID = 1L;
-
- protected Map<String, Object> configuration;
- protected transient AlgebricksPartitionConstraint partitionConstraint;
- protected IAType atype;
- protected IHyracksTaskContext ctx;
- protected AdapterType adapterType;
-
- protected static final HashMap<ATypeTag, IValueParserFactory> typeToValueParserFactMap = new HashMap<ATypeTag, IValueParserFactory>();
- static {
- typeToValueParserFactMap.put(ATypeTag.INT32, IntegerParserFactory.INSTANCE);
- typeToValueParserFactMap.put(ATypeTag.FLOAT, FloatParserFactory.INSTANCE);
- typeToValueParserFactMap.put(ATypeTag.DOUBLE, DoubleParserFactory.INSTANCE);
- typeToValueParserFactMap.put(ATypeTag.INT64, LongParserFactory.INSTANCE);
- typeToValueParserFactMap.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
- }
-
- protected static final Map<String, Object> formatToParserFactoryMap = initializeFormatParserFactoryMap();
-
- public static final String KEY_FORMAT = "format";
- public static final String KEY_PARSER_FACTORY = "parser";
- public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
- public static final String FORMAT_ADM = "adm";
-
- private static Map<String, Object> initializeFormatParserFactoryMap() {
- Map<String, Object> map = new HashMap<String, Object>();
- map.put(FORMAT_DELIMITED_TEXT, "edu.uci.ics.asterix.runtime.operators.file.NtDelimitedDataTupleParserFactory");
- map.put(FORMAT_ADM, "edu.uci.ics.asterix.runtime.operators.file.AdmSchemafullRecordParserFactory");
- return map;
- }
-
- /**
- * Get the partition constraint chosen by the adapter.
- * An adapter may have preferences as to where it needs to be instantiated and used.
- */
- public abstract AlgebricksPartitionConstraint getPartitionConstraint() throws Exception;
-
- /**
- * Get the configured value from the adapter configuration parameters, corresponding to the an attribute.
- *
- * @param attribute
- * The attribute whose value needs to be obtained.
- */
- public Object getAdapterProperty(String attribute) {
- return configuration.get(attribute);
- }
-
- /**
- * Get the adapter configuration parameters.
- *
- * @return A Map<String,String> instance representing the adapter configuration.
- */
- public Map<String, Object> getConfiguration() {
- return configuration;
- }
-
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/CNNFeedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/CNNFeedAdapter.java
index 9112aae..017b511 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/CNNFeedAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/CNNFeedAdapter.java
@@ -14,94 +14,25 @@
*/
package edu.uci.ics.asterix.external.dataset.adapter;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
import java.util.Map;
-import edu.uci.ics.asterix.feed.managed.adapter.IManagedFeedAdapter;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
/**
* An Adapter that provides the functionality of fetching news feed from CNN service
* The Adapter provides news feed as ADM records.
*/
-public class CNNFeedAdapter extends RSSFeedAdapter implements IDatasourceAdapter, IManagedFeedAdapter {
+public class CNNFeedAdapter extends RSSFeedAdapter implements IDatasourceAdapter, IFeedAdapter {
- private static final long serialVersionUID = 2523303758114582251L;
- private List<String> feedURLs = new ArrayList<String>();
- private static Map<String, String> topicFeeds = new HashMap<String, String>();
+ private static final long serialVersionUID = 1L;
- public static final String KEY_RSS_URL = "topic";
- public static final String KEY_INTERVAL = "interval";
- public static final String TOP_STORIES = "topstories";
- public static final String WORLD = "world";
- public static final String US = "us";
- public static final String SPORTS = "sports";
- public static final String BUSINESS = "business";
- public static final String POLITICS = "politics";
- public static final String CRIME = "crime";
- public static final String TECHNOLOGY = "technology";
- public static final String HEALTH = "health";
- public static final String ENTERNTAINMENT = "entertainemnt";
- public static final String TRAVEL = "travel";
- public static final String LIVING = "living";
- public static final String VIDEO = "video";
- public static final String STUDENT = "student";
- public static final String POPULAR = "popular";
- public static final String RECENT = "recent";
-
- private void initTopics() {
- topicFeeds.put(TOP_STORIES, "http://rss.cnn.com/rss/cnn_topstories.rss");
- topicFeeds.put(WORLD, "http://rss.cnn.com/rss/cnn_world.rss");
- topicFeeds.put(US, "http://rss.cnn.com/rss/cnn_us.rss");
- topicFeeds.put(SPORTS, "http://rss.cnn.com/rss/si_topstories.rss");
- topicFeeds.put(BUSINESS, "http://rss.cnn.com/rss/money_latest.rss");
- topicFeeds.put(POLITICS, "http://rss.cnn.com/rss/cnn_allpolitics.rss");
- topicFeeds.put(CRIME, "http://rss.cnn.com/rss/cnn_crime.rss");
- topicFeeds.put(TECHNOLOGY, "http://rss.cnn.com/rss/cnn_tech.rss");
- topicFeeds.put(HEALTH, "http://rss.cnn.com/rss/cnn_health.rss");
- topicFeeds.put(ENTERNTAINMENT, "http://rss.cnn.com/rss/cnn_showbiz.rss");
- topicFeeds.put(LIVING, "http://rss.cnn.com/rss/cnn_living.rss");
- topicFeeds.put(VIDEO, "http://rss.cnn.com/rss/cnn_freevideo.rss");
- topicFeeds.put(TRAVEL, "http://rss.cnn.com/rss/cnn_travel.rss");
- topicFeeds.put(STUDENT, "http://rss.cnn.com/rss/cnn_studentnews.rss");
- topicFeeds.put(POPULAR, "http://rss.cnn.com/rss/cnn_mostpopular.rss");
- topicFeeds.put(RECENT, "http://rss.cnn.com/rss/cnn_latest.rss");
- }
-
- @Override
- public void configure(Map<String, Object> arguments) throws Exception {
- configuration = arguments;
- String rssURLProperty = (String) configuration.get(KEY_RSS_URL);
- if (rssURLProperty == null) {
- throw new IllegalArgumentException("no rss url provided");
- }
- initializeFeedURLs(rssURLProperty);
- configurePartitionConstraints();
-
- }
-
- private void initializeFeedURLs(String rssURLProperty) {
- feedURLs.clear();
- String[] rssTopics = rssURLProperty.split(",");
- initTopics();
- for (String topic : rssTopics) {
- String feedURL = topicFeeds.get(topic);
- if (feedURL == null) {
- throw new IllegalArgumentException(" unknown topic :" + topic + " please choose from the following "
- + getValidTopics());
- }
- feedURLs.add(feedURL);
- }
- }
-
- private static String getValidTopics() {
- StringBuilder builder = new StringBuilder();
- for (String key : topicFeeds.keySet()) {
- builder.append(key);
- builder.append(" ");
- }
- return new String(builder);
+ public CNNFeedAdapter(Map<String, String> configuration, ARecordType recordType, IHyracksTaskContext ctx)
+ throws AsterixException {
+ super(configuration, recordType, ctx);
}
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java
index 33ee11f..45a827c 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java
@@ -16,158 +16,37 @@
import java.io.IOException;
import java.io.InputStream;
-import java.util.List;
-import java.util.Map;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.external.util.DNSResolverFactory;
-import edu.uci.ics.asterix.external.util.INodeResolver;
-import edu.uci.ics.asterix.external.util.INodeResolverFactory;
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.asterix.runtime.operators.file.AdmSchemafullRecordParserFactory;
-import edu.uci.ics.asterix.runtime.operators.file.NtDelimitedDataTupleParserFactory;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
-public abstract class FileSystemBasedAdapter extends AbstractDatasourceAdapter {
+public abstract class FileSystemBasedAdapter implements IDatasourceAdapter {
private static final long serialVersionUID = 1L;
public static final String NODE_RESOLVER_FACTORY_PROPERTY = "node.Resolver";
- public static final String KEY_DELIMITER = "delimiter";
- public static final String KEY_PATH = "path";
-
- protected ITupleParserFactory parserFactory;
- protected ITupleParser parser;
- protected static INodeResolver nodeResolver;
-
- private static final INodeResolver DEFAULT_NODE_RESOLVER = new DNSResolverFactory().createNodeResolver();
- private static final Logger LOGGER = Logger.getLogger(FileSystemBasedAdapter.class.getName());
public abstract InputStream getInputStream(int partition) throws IOException;
- public FileSystemBasedAdapter(IAType atype) {
- this.atype = atype;
+ protected final ITupleParser tupleParser;
+ protected final IAType sourceDatatype;
+ protected IHyracksTaskContext ctx;
+
+ public FileSystemBasedAdapter(ITupleParserFactory parserFactory, IAType sourceDatatype, IHyracksTaskContext ctx) throws HyracksDataException {
+ this.tupleParser = parserFactory.createTupleParser(ctx);
+ this.sourceDatatype = sourceDatatype;
+ this.ctx = ctx;
}
@Override
public void start(int partition, IFrameWriter writer) throws Exception {
InputStream in = getInputStream(partition);
- parser = getTupleParser();
- parser.parse(in, writer);
+ tupleParser.parse(in, writer);
}
- @Override
- public abstract void initialize(IHyracksTaskContext ctx) throws Exception;
-
- @Override
- public abstract void configure(Map<String, Object> arguments) throws Exception;
-
- @Override
- public abstract AdapterType getAdapterType();
-
- @Override
- public abstract AlgebricksPartitionConstraint getPartitionConstraint() throws Exception;
-
- protected ITupleParser getTupleParser() throws Exception {
- return parserFactory.createTupleParser(ctx);
- }
-
- protected void configureFormat() throws Exception {
- String parserFactoryClassname = (String) configuration.get(KEY_PARSER_FACTORY);
- if (parserFactoryClassname == null) {
- String specifiedFormat = (String) configuration.get(KEY_FORMAT);
- if (specifiedFormat == null) {
- throw new IllegalArgumentException(" Unspecified data format");
- } else if (FORMAT_DELIMITED_TEXT.equalsIgnoreCase(specifiedFormat)) {
- parserFactory = getDelimitedDataTupleParserFactory((ARecordType) atype);
- } else if (FORMAT_ADM.equalsIgnoreCase((String) configuration.get(KEY_FORMAT))) {
- parserFactory = getADMDataTupleParserFactory((ARecordType) atype);
- } else {
- throw new IllegalArgumentException(" format " + configuration.get(KEY_FORMAT) + " not supported");
- }
- } else {
- parserFactory = (ITupleParserFactory) Class.forName(parserFactoryClassname).newInstance();
- }
-
- }
-
- protected ITupleParserFactory getDelimitedDataTupleParserFactory(ARecordType recordType) throws AsterixException {
- int n = recordType.getFieldTypes().length;
- IValueParserFactory[] fieldParserFactories = new IValueParserFactory[n];
- for (int i = 0; i < n; i++) {
- ATypeTag tag = null;
- if (recordType.getFieldTypes()[i].getTypeTag() == ATypeTag.UNION) {
- List<IAType> unionTypes = ((AUnionType) recordType.getFieldTypes()[i]).getUnionList();
- if (unionTypes.size() != 2 && unionTypes.get(0).getTypeTag() != ATypeTag.NULL) {
- throw new NotImplementedException("Non-optional UNION type is not supported.");
- }
- tag = unionTypes.get(1).getTypeTag();
- } else {
- tag = recordType.getFieldTypes()[i].getTypeTag();
- }
- if (tag == null) {
- throw new NotImplementedException("Failed to get the type information for field " + i + ".");
- }
- IValueParserFactory vpf = typeToValueParserFactMap.get(tag);
- if (vpf == null) {
- throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
- }
- fieldParserFactories[i] = vpf;
- }
- String delimiterValue = (String) configuration.get(KEY_DELIMITER);
- if (delimiterValue != null && delimiterValue.length() > 1) {
- throw new AsterixException("improper delimiter");
- }
-
- Character delimiter = delimiterValue.charAt(0);
- return new NtDelimitedDataTupleParserFactory(recordType, fieldParserFactories, delimiter);
- }
-
- protected ITupleParserFactory getADMDataTupleParserFactory(ARecordType recordType) throws AsterixException {
- try {
- return new AdmSchemafullRecordParserFactory(recordType);
- } catch (Exception e) {
- throw new AsterixException(e);
- }
-
- }
-
- protected INodeResolver getNodeResolver() {
- if (nodeResolver == null) {
- nodeResolver = initNodeResolver();
- }
- return nodeResolver;
- }
-
- private static INodeResolver initNodeResolver() {
- INodeResolver nodeResolver = null;
- String configuredNodeResolverFactory = System.getProperty(NODE_RESOLVER_FACTORY_PROPERTY);
- if (configuredNodeResolverFactory != null) {
- try {
- nodeResolver = ((INodeResolverFactory) (Class.forName(configuredNodeResolverFactory).newInstance()))
- .createNodeResolver();
-
- } catch (Exception e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.log(Level.WARNING, "Unable to create node resolver from the configured classname "
- + configuredNodeResolverFactory + "\n" + e.getMessage());
- }
- nodeResolver = DEFAULT_NODE_RESOLVER;
- }
- } else {
- nodeResolver = DEFAULT_NODE_RESOLVER;
- }
- return nodeResolver;
- }
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java
index f8b381b..1e694e2 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java
@@ -16,7 +16,6 @@
import java.io.IOException;
import java.io.InputStream;
-import java.util.Map;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.Counters.Counter;
@@ -31,6 +30,8 @@
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
/**
* Provides functionality for fetching external data stored in an HDFS instance.
@@ -44,34 +45,16 @@
private transient boolean executed[];
private transient InputSplit[] inputSplits;
private transient JobConf conf;
- private transient AlgebricksPartitionConstraint clusterLocations;
-
private transient String nodeName;
public HDFSAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
- AlgebricksPartitionConstraint clusterLocations) {
- super(atype);
+ String nodeName, ITupleParserFactory parserFactory, IHyracksTaskContext ctx) throws HyracksDataException {
+ super(parserFactory, atype, ctx);
this.readSchedule = readSchedule;
this.executed = executed;
this.inputSplits = inputSplits;
this.conf = conf;
- this.clusterLocations = clusterLocations;
- }
-
- @Override
- public void configure(Map<String, Object> arguments) throws Exception {
- this.configuration = arguments;
- configureFormat();
- }
-
- public AdapterType getAdapterType() {
- return AdapterType.READ_WRITE;
- }
-
- @Override
- public void initialize(IHyracksTaskContext ctx) throws Exception {
- this.ctx = ctx;
- this.nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+ this.nodeName = nodeName;
}
private Reporter getReporter() {
@@ -227,9 +210,4 @@
}
- @Override
- public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
- return clusterLocations;
- }
-
}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveAdapter.java
index 91cf0d2..f4ff44e 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveAdapter.java
@@ -14,21 +14,23 @@
*/
package edu.uci.ics.asterix.external.dataset.adapter;
-import java.util.Map;
+import java.io.IOException;
+import java.io.InputStream;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
/**
* Provides the functionality of fetching data in form of ADM records from a Hive dataset.
*/
@SuppressWarnings("deprecation")
-public class HiveAdapter extends AbstractDatasourceAdapter {
+public class HiveAdapter extends FileSystemBasedAdapter {
private static final long serialVersionUID = 1L;
@@ -42,35 +44,16 @@
private HDFSAdapter hdfsAdapter;
public HiveAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
- AlgebricksPartitionConstraint clusterLocations) {
- this.hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
- this.atype = atype;
+ AlgebricksPartitionConstraint clusterLocations, String nodeName, ITupleParserFactory parserFactory,
+ IHyracksTaskContext ctx) throws HyracksDataException {
+ super(parserFactory, atype, ctx);
+ this.hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, nodeName, parserFactory,
+ ctx);
}
@Override
- public AdapterType getAdapterType() {
- return AdapterType.READ;
- }
-
- @Override
- public void configure(Map<String, Object> arguments) throws Exception {
- this.configuration = arguments;
- this.hdfsAdapter.configure(arguments);
- }
-
- @Override
- public void initialize(IHyracksTaskContext ctx) throws Exception {
- hdfsAdapter.initialize(ctx);
- }
-
- @Override
- public void start(int partition, IFrameWriter writer) throws Exception {
- hdfsAdapter.start(partition, writer);
- }
-
- @Override
- public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
- return hdfsAdapter.getPartitionConstraint();
+ public InputStream getInputStream(int partition) throws IOException {
+ return hdfsAdapter.getInputStream(partition);
}
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IDatasourceAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IDatasourceAdapter.java
deleted file mode 100644
index 9403bfe..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IDatasourceAdapter.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.external.dataset.adapter;
-
-import java.io.Serializable;
-import java.util.Map;
-
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-
-/**
- * A super interface implemented by a data source adapter. An adapter can be a
- * pull based or push based. This interface provides all common APIs that need
- * to be implemented by each adapter irrespective of the the kind of
- * adapter(pull or push).
- */
-public interface IDatasourceAdapter extends Serializable {
-
- /**
- * An adapter can be used to read from an external data source and may also
- * allow writing to the external data source. This enum type indicates the
- * kind of operations supported by the adapter.
- *
- * @caller Compiler uses this method to assert the validity of an operation
- * on an external dataset. The type of adapter associated with an
- * external dataset determines the set of valid operations allowed
- * on the dataset.
- */
- public enum AdapterType {
- READ,
- WRITE,
- READ_WRITE
- }
-
- /**
- * Returns the type of adapter indicating if the adapter can be used for
- * reading from an external data source or writing to an external data
- * source or can be used for both purposes.
- *
- * @Caller: Compiler: The compiler uses this API to verify if an operation
- * is supported by the adapter. For example, an write query against
- * an external dataset will not compile successfully if the
- * external dataset was declared with a read_only adapter.
- * @see AdapterType
- * @return
- */
- public AdapterType getAdapterType();
-
- /**
- * Each adapter instance is configured with a set of parameters that are
- * key-value pairs. When creating an external or a feed dataset, an adapter
- * instance is used in conjunction with a set of configuration parameters
- * for the adapter instance. The configuration parameters are stored
- * internally with the adapter and can be retrieved using this API.
- *
- * @param propertyKey
- * @return String the value corresponding to the configuration parameter
- * represented by the key- attributeKey.
- */
- public Object getAdapterProperty(String propertyKey);
-
- /**
- * Configures the IDatasourceAdapter instance.
- *
- * @caller Scenario 1) Called during compilation of DDL statement that
- * creates a Feed dataset and associates the adapter with the
- * dataset. The (key,value) configuration parameters provided as
- * part of the DDL statement are collected by the compiler and
- * passed on to this method. The adapter may as part of
- * configuration connect with the external data source and determine
- * the IAType associated with data residing with the external
- * datasource.
- * Scenario 2) An adapter instance is created by an ASTERIX operator
- * that wraps around the adapter instance. The operator, as part of
- * its initialization invokes the configure method. The (key,value)
- * configuration parameters are passed on to the operator by the
- * compiler. Subsequent to the invocation, the wrapping operator
- * obtains the partition constraints (if any). In addition, in the
- * case of a read adapter, the wrapping operator obtains the output
- * ASTERIX type associated with the data that will be output from
- * the adapter.
- * @param arguments
- * A map with key-value pairs that contains the configuration
- * parameters for the adapter. The arguments are obtained from
- * the metadata. Recall that the DDL to create an external
- * dataset or a feed dataset requires using an adapter and
- * providing all arguments as a set of (key,value) pairs. These
- * arguments are put into the metadata.
- */
- public void configure(Map<String, Object> arguments) throws Exception;
-
- /**
- * Returns a list of partition constraints. A partition constraint can be a
- * requirement to execute at a particular location or could be cardinality
- * constraints indicating the number of instances that need to run in
- * parallel. example, a IDatasourceAdapter implementation written for data
- * residing on the local file system of a node cannot run on any other node
- * and thus has a location partition constraint. The location partition
- * constraint can be expressed as a node IP address or a node controller id.
- * In the former case, the IP address is translated to a node controller id
- * running on the node with the given IP address.
- *
- * @Caller The wrapper operator configures its partition constraints from
- * the constraints obtained from the adapter.
- */
- public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception;
-
- /**
- * Allows the adapter to establish connection with the external data source
- * expressing intent for data and providing any configuration parameters
- * required by the external data source for the transfer of data. This
- * method does not result in any data transfer, but is a prerequisite for
- * any subsequent data transfer to happen between the external data source
- * and the adapter.
- *
- * @caller This method is called by the wrapping ASTERIX operator that
- * @param ctx
- * @throws Exception
- */
- public void initialize(IHyracksTaskContext ctx) throws Exception;
-
- /**
- * Triggers the adapter to begin ingestion of data from the external source.
- *
- * @param partition
- * The adapter could be running with a degree of parallelism.
- * partition corresponds to the i'th parallel instance.
- * @param writer
- * The instance of frame writer that is used by the adapter to
- * write frame to. Adapter packs the fetched bytes (from external source),
- * packs them into frames and forwards the frames to an upstream receiving
- * operator using the instance of IFrameWriter.
- * @throws Exception
- */
- public void start(int partition, IFrameWriter writer) throws Exception;
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IFeedClientFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IFeedClientFactory.java
new file mode 100644
index 0000000..4fb14ff
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IFeedClientFactory.java
@@ -0,0 +1,22 @@
+package edu.uci.ics.asterix.external.dataset.adapter;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+public interface IFeedClientFactory {
+
+ public IPullBasedFeedClient createFeedClient(IHyracksTaskContext ctx, Map<String, String> configuration)
+ throws Exception;
+
+ public ARecordType getRecordType() throws AsterixException;
+
+ public FeedClientType getFeedClientType();
+
+ public enum FeedClientType {
+ GENERIC,
+ TYPED
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IPullBasedFeedClient.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IPullBasedFeedClient.java
index aeecb5f..ee28c3a 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IPullBasedFeedClient.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IPullBasedFeedClient.java
@@ -15,11 +15,18 @@
package edu.uci.ics.asterix.external.dataset.adapter;
import java.io.DataOutput;
+import java.util.Map;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
public interface IPullBasedFeedClient {
+ public enum InflowState {
+ NO_MORE_DATA,
+ DATA_AVAILABLE,
+ DATA_NOT_AVAILABLE
+ }
+
/**
* Writes the next fetched tuple into the provided instance of DatatOutput.
*
@@ -29,7 +36,7 @@
* false if no record was written to the DataOutput instance indicating non-availability of new data.
* @throws AsterixException
*/
- public boolean nextTuple(DataOutput dataOutput) throws AsterixException;
+ public InflowState nextTuple(DataOutput dataOutput) throws AsterixException;
/**
* Provides logic for any corrective action that feed client needs to execute on
@@ -42,10 +49,10 @@
public void resetOnFailure(Exception e) throws AsterixException;
/**
- * Terminates a feed, that is data ingestion activity ceases.
- *
- * @throws Exception
+ * @param configuration
*/
- public void stop() throws Exception;
+ public boolean alter(Map<String, String> configuration);
+
+ public void stop();
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IPushBasedFeedClient.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IPushBasedFeedClient.java
new file mode 100644
index 0000000..da49240
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IPushBasedFeedClient.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.external.dataset.adapter;
+
+import java.io.InputStream;
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+
+public interface IPushBasedFeedClient {
+
+ /**
+ * @return
+ * @throws AsterixException
+ */
+ public InputStream getInputStream() throws AsterixException;
+
+ /**
+ * Provides logic for any corrective action that feed client needs to execute on
+ * encountering an exception.
+ *
+ * @param e
+ * The exception encountered during fetching of data from external source
+ * @throws AsterixException
+ */
+ public void resetOnFailure(Exception e) throws AsterixException;
+
+ /**
+ * @param configuration
+ */
+ public boolean alter(Map<String, Object> configuration);
+
+ public void stop();
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/NCFileSystemAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/NCFileSystemAdapter.java
index 182ccf7..ea8ed05 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/NCFileSystemAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/NCFileSystemAdapter.java
@@ -19,15 +19,12 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
-import java.util.Map;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
/**
* Factory class for creating an instance of NCFileSystemAdapter. An
@@ -37,59 +34,13 @@
public class NCFileSystemAdapter extends FileSystemBasedAdapter {
private static final long serialVersionUID = 1L;
- private FileSplit[] fileSplits;
- public NCFileSystemAdapter(IAType atype) {
- super(atype);
- }
+ private final FileSplit[] fileSplits;
- @Override
- public void configure(Map<String, Object> arguments) throws Exception {
- this.configuration = arguments;
- String[] splits = ((String) arguments.get(KEY_PATH)).split(",");
- configureFileSplits(splits);
- configureFormat();
- }
-
- @Override
- public void initialize(IHyracksTaskContext ctx) throws Exception {
- this.ctx = ctx;
- }
-
- @Override
- public AdapterType getAdapterType() {
- return AdapterType.READ;
- }
-
- private void configureFileSplits(String[] splits) throws AsterixException {
- if (fileSplits == null) {
- fileSplits = new FileSplit[splits.length];
- String nodeName;
- String nodeLocalPath;
- int count = 0;
- String trimmedValue;
- for (String splitPath : splits) {
- trimmedValue = splitPath.trim();
- if (!trimmedValue.contains("://")) {
- throw new AsterixException("Invalid path: " + splitPath
- + "\nUsage- path=\"Host://Absolute File Path\"");
- }
- nodeName = trimmedValue.split(":")[0];
- nodeLocalPath = trimmedValue.split("://")[1];
- FileSplit fileSplit = new FileSplit(nodeName, new FileReference(new File(nodeLocalPath)));
- fileSplits[count++] = fileSplit;
- }
- }
- }
-
- private void configurePartitionConstraint() throws AsterixException {
- String[] locs = new String[fileSplits.length];
- String location;
- for (int i = 0; i < fileSplits.length; i++) {
- location = getNodeResolver().resolveNode(fileSplits[i].getNodeName());
- locs[i] = location;
- }
- partitionConstraint = new AlgebricksAbsolutePartitionConstraint(locs);
+ public NCFileSystemAdapter(FileSplit[] fileSplits, ITupleParserFactory parserFactory, IAType atype,
+ IHyracksTaskContext ctx) throws HyracksDataException {
+ super(parserFactory, atype, ctx);
+ this.fileSplits = fileSplits;
}
@Override
@@ -105,11 +56,4 @@
}
}
- @Override
- public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
- if (partitionConstraint == null) {
- configurePartitionConstraint();
- }
- return partitionConstraint;
- }
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedAdapter.java
index 93d2057..da2264d 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedAdapter.java
@@ -15,10 +15,17 @@
package edu.uci.ics.asterix.external.dataset.adapter;
import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.external.dataset.adapter.IPullBasedFeedClient.InflowState;
+import edu.uci.ics.asterix.metadata.feeds.AbstractFeedDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -29,18 +36,39 @@
* Captures the common logic for obtaining bytes from an external source
* and packing them into frames as tuples.
*/
-public abstract class PullBasedAdapter extends AbstractDatasourceAdapter implements ITypedDatasourceAdapter {
+public abstract class PullBasedAdapter extends AbstractFeedDatasourceAdapter implements IDatasourceAdapter,
+ IFeedAdapter {
private static final long serialVersionUID = 1L;
+ private static final Logger LOGGER = Logger.getLogger(PullBasedAdapter.class.getName());
protected ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(1);
protected IPullBasedFeedClient pullBasedFeedClient;
protected ARecordType adapterOutputType;
private FrameTupleAppender appender;
private ByteBuffer frame;
+ protected boolean continueIngestion = true;
+ protected boolean alterRequested = false;
+ private Map<String, String> modifiedConfiguration = null;
+ private long tupleCount = 0;
+ private final IHyracksTaskContext ctx;
+ protected Map<String, String> configuration;
public abstract IPullBasedFeedClient getFeedClient(int partition) throws Exception;
+ public PullBasedAdapter(Map<String, String> configuration, IHyracksTaskContext ctx) {
+ this.ctx = ctx;
+ this.configuration = configuration;
+ }
+
+ public long getIngestedRecordsCount() {
+ return tupleCount;
+ }
+
+ public void alter(Map<String, String> modifedConfiguration) {
+ this.modifiedConfiguration = modifedConfiguration;
+ }
+
@Override
public void start(int partition, IFrameWriter writer) throws Exception {
appender = new FrameTupleAppender(ctx.getFrameSize());
@@ -48,22 +76,45 @@
appender.reset(frame, true);
pullBasedFeedClient = getFeedClient(partition);
- boolean moreData = false;
- while (true) {
+ InflowState inflowState = null;
+ while (continueIngestion) {
tupleBuilder.reset();
try {
- moreData = pullBasedFeedClient.nextTuple(tupleBuilder.getDataOutput());
- if (moreData) {
- tupleBuilder.addFieldEndOffset();
- appendTupleToFrame(writer);
- } else {
- FrameUtils.flushFrame(frame, writer);
- break;
+ inflowState = pullBasedFeedClient.nextTuple(tupleBuilder.getDataOutput());
+ switch (inflowState) {
+ case DATA_AVAILABLE:
+ tupleBuilder.addFieldEndOffset();
+ appendTupleToFrame(writer);
+ tupleCount++;
+ break;
+ case NO_MORE_DATA:
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Reached end of feed");
+ }
+ FrameUtils.flushFrame(frame, writer);
+ continueIngestion = false;
+ break;
+ case DATA_NOT_AVAILABLE:
+ break;
+ }
+ if (alterRequested) {
+ boolean success = pullBasedFeedClient.alter(modifiedConfiguration);
+ if (success) {
+ configuration = modifiedConfiguration;
+ modifiedConfiguration = null;
+ }
}
} catch (Exception failureException) {
try {
- pullBasedFeedClient.resetOnFailure(failureException);
- continue;
+ failureException.printStackTrace();
+ boolean continueIngestion = policyEnforcer.handleSoftwareFailure(failureException);
+ if (continueIngestion) {
+ pullBasedFeedClient.resetOnFailure(failureException);
+ tupleBuilder.reset();
+ continue;
+ } else {
+ throw failureException;
+ }
} catch (Exception recoveryException) {
throw new Exception(recoveryException);
}
@@ -71,19 +122,11 @@
}
}
- /**
- * Allows an adapter to handle a runtime exception.
- * @param e exception encountered during runtime
- * @throws AsterixException
- */
- public void resetOnFailure(Exception e) throws AsterixException {
- pullBasedFeedClient.resetOnFailure(e);
- tupleBuilder.reset();
- }
-
private void appendTupleToFrame(IFrameWriter writer) throws HyracksDataException {
if (!appender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0, tupleBuilder.getSize())) {
+ System.out.println("flushing frame");
FrameUtils.flushFrame(frame, writer);
+ System.out.println("flushed frame");
appender.reset(frame, true);
if (!appender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
tupleBuilder.getSize())) {
@@ -92,9 +135,17 @@
}
}
- @Override
- public ARecordType getAdapterOutputType() {
- return adapterOutputType;
+ /**
+ * Discontinue the ingestion of data and end the feed.
+ *
+ * @throws Exception
+ */
+ public void stop() throws Exception {
+ continueIngestion = false;
+ }
+
+ public Map<String, String> getConfiguration() {
+ return configuration;
}
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedFeedClient.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedFeedClient.java
index 6d05bde..8efe919 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedFeedClient.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedFeedClient.java
@@ -17,46 +17,138 @@
import java.io.DataOutput;
import java.io.IOException;
+import edu.uci.ics.asterix.builders.IARecordBuilder;
+import edu.uci.ics.asterix.builders.RecordBuilder;
+import edu.uci.ics.asterix.builders.UnorderedListBuilder;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ABoolean;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AMutableDateTime;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutablePoint;
import edu.uci.ics.asterix.om.base.AMutableRecord;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.AMutableUnorderedList;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.base.IACursor;
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
public abstract class PullBasedFeedClient implements IPullBasedFeedClient {
protected ARecordSerializerDeserializer recordSerDe;
protected AMutableRecord mutableRecord;
protected boolean messageReceived;
- protected boolean continueIngestion=true;
+ protected boolean continueIngestion = true;
+ protected IARecordBuilder recordBuilder = new RecordBuilder();
- public abstract boolean setNextRecord() throws Exception;
+ protected AMutableString aString = new AMutableString("");
+ protected AMutableInt32 aInt32 = new AMutableInt32(0);
+ protected AMutablePoint aPoint = new AMutablePoint(0, 0);
+ protected AMutableDateTime aDateTime = new AMutableDateTime(0);
+
+ @SuppressWarnings("unchecked")
+ protected ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ASTRING);
+ @SuppressWarnings("unchecked")
+ protected ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ABOOLEAN);
+ @SuppressWarnings("unchecked")
+ protected ISerializerDeserializer<AInt32> int32Serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT32);
+
+ public abstract InflowState setNextRecord() throws Exception;
@Override
- public boolean nextTuple(DataOutput dataOutput) throws AsterixException {
+ public InflowState nextTuple(DataOutput dataOutput) throws AsterixException {
try {
- boolean newData = setNextRecord();
- if (newData && continueIngestion) {
- IAType t = mutableRecord.getType();
- ATypeTag tag = t.getTypeTag();
- try {
+ System.out.println("Setting next record");
+ InflowState state = setNextRecord();
+ boolean first = true;
+ switch (state) {
+ case DATA_AVAILABLE:
+ IAType t = mutableRecord.getType();
+ ATypeTag tag = t.getTypeTag();
dataOutput.writeByte(tag.serialize());
- } catch (IOException e) {
- throw new HyracksDataException(e);
- }
- recordSerDe.serialize(mutableRecord, dataOutput);
- return true;
+ if (first) {
+ recordBuilder.reset(mutableRecord.getType());
+ first = false;
+ }
+ recordBuilder.init();
+ writeRecord(mutableRecord, dataOutput, recordBuilder);
+ break;
+
+ case DATA_NOT_AVAILABLE:
+ break;
+ case NO_MORE_DATA:
+ break;
}
- return false;
+ return state;
} catch (Exception e) {
throw new AsterixException(e);
}
}
- @Override
- public void stop() {
- continueIngestion = false;
+ private void writeRecord(AMutableRecord record, DataOutput dataOutput, IARecordBuilder recordBuilder)
+ throws IOException, AsterixException {
+ ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
+ int numFields = record.getType().getFieldNames().length;
+ for (int pos = 0; pos < numFields; pos++) {
+ fieldValue.reset();
+ IAObject obj = record.getValueByPos(pos);
+ writeObject(obj, fieldValue.getDataOutput());
+ recordBuilder.addField(pos, fieldValue);
+ }
+ recordBuilder.write(dataOutput, false);
+ }
+
+ private void writeObject(IAObject obj, DataOutput dataOutput) throws IOException, AsterixException {
+ switch (obj.getType().getTypeTag()) {
+ case RECORD:
+ ATypeTag tag = obj.getType().getTypeTag();
+ try {
+ dataOutput.writeByte(tag.serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ IARecordBuilder recordBuilder = new RecordBuilder();
+ recordBuilder.reset((ARecordType) obj.getType());
+ recordBuilder.init();
+ writeRecord((AMutableRecord) obj, dataOutput, recordBuilder);
+ break;
+ case UNORDEREDLIST:
+ tag = obj.getType().getTypeTag();
+ try {
+ dataOutput.writeByte(tag.serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ UnorderedListBuilder listBuilder = new UnorderedListBuilder();
+ listBuilder.reset((AUnorderedListType) ((AMutableUnorderedList) obj).getType());
+ IACursor cursor = ((AMutableUnorderedList) obj).getCursor();
+ ArrayBackedValueStorage listItemValue = new ArrayBackedValueStorage();
+ while (cursor.next()) {
+ listItemValue.reset();
+ IAObject item = cursor.get();
+ writeObject(item, listItemValue.getDataOutput());
+ listBuilder.addItem(listItemValue);
+ }
+ listBuilder.write(dataOutput, false);
+ break;
+ default:
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(obj.getType()).serialize(obj,
+ dataOutput);
+ break;
+ }
}
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterAdapter.java
index 2715a00..41143d8 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterAdapter.java
@@ -14,32 +14,28 @@
*/
package edu.uci.ics.asterix.external.dataset.adapter;
-import java.util.HashMap;
import java.util.Map;
-import edu.uci.ics.asterix.feed.managed.adapter.IManagedFeedAdapter;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
/**
* An adapter that provides the functionality of receiving tweets from the
* Twitter service in the form of ADM formatted records.
*/
-public class PullBasedTwitterAdapter extends PullBasedAdapter implements IManagedFeedAdapter {
+public class PullBasedTwitterAdapter extends PullBasedAdapter implements IFeedAdapter {
private static final long serialVersionUID = 1L;
public static final String QUERY = "query";
public static final String INTERVAL = "interval";
- private boolean alterRequested = false;
- private Map<String, String> alteredParams = new HashMap<String, String>();
private ARecordType recordType;
-
+ private final IHyracksTaskContext ctx;
private PullBasedTwitterFeedClient tweetClient;
@Override
@@ -47,27 +43,13 @@
return tweetClient;
}
- @Override
- public void configure(Map<String, Object> arguments) throws Exception {
- configuration = arguments;
- String[] fieldNames = { "id", "username", "location", "text", "timestamp" };
- IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
- BuiltinType.ASTRING };
- recordType = new ARecordType("FeedRecordType", fieldNames, fieldTypes, false);
- }
-
- @Override
- public void initialize(IHyracksTaskContext ctx) throws Exception {
+ public PullBasedTwitterAdapter(Map<String, String> configuration, IHyracksTaskContext ctx) throws AsterixException {
+ super(configuration, ctx);
this.ctx = ctx;
tweetClient = new PullBasedTwitterFeedClient(ctx, this);
}
@Override
- public AdapterType getAdapterType() {
- return AdapterType.READ;
- }
-
- @Override
public void stop() {
tweetClient.stop();
}
@@ -75,33 +57,18 @@
@Override
public void alter(Map<String, String> properties) {
alterRequested = true;
- this.alteredParams = properties;
}
public boolean isAlterRequested() {
return alterRequested;
}
- public Map<String, String> getAlteredParams() {
- return alteredParams;
- }
-
public void postAlteration() {
- alteredParams = null;
alterRequested = false;
}
- @Override
public ARecordType getAdapterOutputType() {
return recordType;
}
- @Override
- public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
- if (partitionConstraint == null) {
- partitionConstraint = new AlgebricksCountPartitionConstraint(1);
- }
- return partitionConstraint;
- }
-
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterFeedClient.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterFeedClient.java
index 9f44a03..7a5aeea 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterFeedClient.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterFeedClient.java
@@ -14,9 +14,10 @@
*/
package edu.uci.ics.asterix.external.dataset.adapter;
-import java.util.LinkedList;
+import java.util.List;
import java.util.Map;
-import java.util.Queue;
+import java.util.UUID;
+import java.util.logging.Logger;
import twitter4j.Query;
import twitter4j.QueryResult;
@@ -41,15 +42,17 @@
private String keywords;
private Query query;
- private long id = 0;
private String id_prefix;
private Twitter twitter;
private int requestInterval = 10; // seconds
- private Queue<Tweet> tweetBuffer = new LinkedList<Tweet>();
+ private QueryResult result;
- IAObject[] mutableFields;
- String[] tupleFieldValues;
+ private IAObject[] mutableFields;
+ private String[] tupleFieldValues;
private ARecordType recordType;
+ private int nextTweetIndex = 0;
+
+ private static final Logger LOGGER = Logger.getLogger(PullBasedTwitterFeedClient.class.getName());
public PullBasedTwitterFeedClient(IHyracksTaskContext ctx, PullBasedTwitterAdapter adapter) {
this.id_prefix = ctx.getJobletContext().getApplicationContext().getNodeId();
@@ -59,24 +62,8 @@
recordType = adapter.getAdapterOutputType();
recordSerDe = new ARecordSerializerDeserializer(recordType);
mutableRecord = new AMutableRecord(recordType, mutableFields);
- initialize(adapter.getConfiguration());
tupleFieldValues = new String[recordType.getFieldNames().length];
- }
-
- public void initialize(Map<String, Object> params) {
- this.keywords = (String) params.get(PullBasedTwitterAdapter.QUERY);
- this.query = new Query(keywords);
- query.setRpp(100);
- }
-
- private Tweet getNextTweet() throws TwitterException, InterruptedException {
- if (tweetBuffer.isEmpty()) {
- QueryResult result;
- Thread.sleep(1000 * requestInterval);
- result = twitter.search(query);
- tweetBuffer.addAll(result.getTweets());
- }
- return tweetBuffer.remove();
+ initialize(adapter.getConfiguration());
}
public ARecordType getRecordType() {
@@ -88,15 +75,14 @@
}
@Override
- public boolean setNextRecord() throws Exception {
+ public InflowState setNextRecord() throws Exception {
Tweet tweet;
tweet = getNextTweet();
if (tweet == null) {
- return false;
+ return InflowState.DATA_NOT_AVAILABLE;
}
int numFields = recordType.getFieldNames().length;
-
- tupleFieldValues[0] = id_prefix + ":" + id;
+ tupleFieldValues[0] = UUID.randomUUID().toString();
tupleFieldValues[1] = tweet.getFromUser();
tupleFieldValues[2] = tweet.getLocation() == null ? "" : tweet.getLocation();
tupleFieldValues[3] = tweet.getText();
@@ -105,8 +91,7 @@
((AMutableString) mutableFields[i]).setValue(tupleFieldValues[i]);
mutableRecord.setValueAtPos(i, mutableFields[i]);
}
- id++;
- return true;
+ return InflowState.DATA_AVAILABLE;
}
@Override
@@ -114,4 +99,32 @@
// TOOO: implement resetting logic for Twitter
}
+ @Override
+ public boolean alter(Map<String, String> configuration) {
+ // TODO Auto-generated method stub
+ return false;
+ }
+
+ @Override
+ public void stop() {
+ // TODO Auto-generated method stub
+ }
+
+ private void initialize(Map<String, String> params) {
+ this.keywords = (String) params.get(PullBasedTwitterAdapter.QUERY);
+ this.requestInterval = Integer.parseInt((String) params.get(PullBasedTwitterAdapter.INTERVAL));
+ this.query = new Query(keywords);
+ query.setRpp(100);
+ }
+
+ private Tweet getNextTweet() throws TwitterException, InterruptedException {
+ if (result == null || nextTweetIndex >= result.getTweets().size()) {
+ Thread.sleep(1000 * requestInterval);
+ result = twitter.search(query);
+ nextTweetIndex = 0;
+ }
+ List<Tweet> tw = result.getTweets();
+ return tw.get(nextTweetIndex++);
+ }
+
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedAdapter.java
index 8379b18..102482d 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedAdapter.java
@@ -19,18 +19,15 @@
import java.util.List;
import java.util.Map;
-import edu.uci.ics.asterix.feed.managed.adapter.IManagedFeedAdapter;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter;
import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.BuiltinType;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
/**
* RSSFeedAdapter provides the functionality of fetching an RSS based feed.
*/
-public class RSSFeedAdapter extends PullBasedAdapter implements IManagedFeedAdapter {
+public class RSSFeedAdapter extends PullBasedAdapter implements IFeedAdapter {
private static final long serialVersionUID = 1L;
@@ -39,17 +36,21 @@
private boolean isAlterRequested = false;
private Map<String, String> alteredParams = new HashMap<String, String>();
private String id_prefix = "";
- private ARecordType recordType;
private IPullBasedFeedClient rssFeedClient;
-
- public static final String KEY_RSS_URL = "url";
- public static final String KEY_INTERVAL = "interval";
+ private ARecordType recordType;
public boolean isStopRequested() {
return isStopRequested;
}
+ public RSSFeedAdapter(Map<String, String> configuration, ARecordType recordType, IHyracksTaskContext ctx)
+ throws AsterixException {
+ super(configuration, ctx);
+ id_prefix = ctx.getJobletContext().getApplicationContext().getNodeId();
+ this.recordType = recordType;
+ }
+
public void setStopRequested(boolean isStopRequested) {
this.isStopRequested = isStopRequested;
}
@@ -66,25 +67,6 @@
isStopRequested = true;
}
- @Override
- public AdapterType getAdapterType() {
- return AdapterType.READ;
- }
-
- @Override
- public void configure(Map<String, Object> arguments) throws Exception {
- configuration = arguments;
- String rssURLProperty = (String) configuration.get(KEY_RSS_URL);
- if (rssURLProperty == null) {
- throw new IllegalArgumentException("no rss url provided");
- }
- initializeFeedURLs(rssURLProperty);
- configurePartitionConstraints();
- recordType = new ARecordType("FeedRecordType", new String[] { "id", "title", "description", "link" },
- new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING },
- false);
- }
-
private void initializeFeedURLs(String rssURLProperty) {
feedURLs.clear();
String[] feedURLProperty = rssURLProperty.split(",");
@@ -94,22 +76,12 @@
}
protected void reconfigure(Map<String, String> arguments) {
- String rssURLProperty = (String) configuration.get(KEY_RSS_URL);
+ String rssURLProperty = configuration.get("KEY_RSS_URL");
if (rssURLProperty != null) {
initializeFeedURLs(rssURLProperty);
}
}
- protected void configurePartitionConstraints() {
- partitionConstraint = new AlgebricksCountPartitionConstraint(feedURLs.size());
- }
-
- @Override
- public void initialize(IHyracksTaskContext ctx) throws Exception {
- this.ctx = ctx;
- id_prefix = ctx.getJobletContext().getApplicationContext().getNodeId();
- }
-
public boolean isAlterRequested() {
return isAlterRequested;
}
@@ -126,17 +98,8 @@
return rssFeedClient;
}
- @Override
- public ARecordType getAdapterOutputType() {
+ public ARecordType getRecordType() {
return recordType;
}
- @Override
- public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
- if (partitionConstraint == null) {
- configurePartitionConstraints();
- }
- return partitionConstraint;
- }
-
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedClient.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedClient.java
index d89674f..8a4b301 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedClient.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedClient.java
@@ -18,6 +18,7 @@
import java.net.URL;
import java.util.LinkedList;
import java.util.List;
+import java.util.Map;
import java.util.Queue;
import com.sun.syndication.feed.synd.SyndEntryImpl;
@@ -75,16 +76,16 @@
fetcher.addFetcherEventListener(listener);
mutableFields = new IAObject[] { new AMutableString(null), new AMutableString(null), new AMutableString(null),
new AMutableString(null) };
- recordType = adapter.getAdapterOutputType();
+ recordType = adapter.getRecordType();
mutableRecord = new AMutableRecord(recordType, mutableFields);
tupleFieldValues = new String[recordType.getFieldNames().length];
}
@Override
- public boolean setNextRecord() throws Exception {
+ public InflowState setNextRecord() throws Exception {
SyndEntryImpl feedEntry = getNextRSSFeed();
if (feedEntry == null) {
- return false;
+ return InflowState.DATA_NOT_AVAILABLE;
}
tupleFieldValues[0] = idPrefix + ":" + id;
tupleFieldValues[1] = feedEntry.getTitle();
@@ -96,7 +97,7 @@
mutableRecord.setValueAtPos(i, mutableFields[i]);
}
id++;
- return true;
+ return InflowState.DATA_AVAILABLE;
}
private SyndEntryImpl getNextRSSFeed() throws Exception {
@@ -138,6 +139,18 @@
}
+ @Override
+ public boolean alter(Map<String, String> configuration) {
+ // TODO Auto-generated method stub
+ return false;
+ }
+
+ @Override
+ public void stop() {
+ // TODO Auto-generated method stub
+
+ }
+
}
class FetcherEventListenerImpl implements FetcherListener {
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/StreamBasedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/StreamBasedAdapter.java
new file mode 100644
index 0000000..f89a7ff
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/StreamBasedAdapter.java
@@ -0,0 +1,40 @@
+package edu.uci.ics.asterix.external.dataset.adapter;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import edu.uci.ics.asterix.metadata.feeds.AdapterRuntimeManager;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+public abstract class StreamBasedAdapter implements IDatasourceAdapter {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final String NODE_RESOLVER_FACTORY_PROPERTY = "node.Resolver";
+
+ public abstract InputStream getInputStream(int partition) throws IOException;
+
+ protected final ITupleParser tupleParser;
+ protected final IAType sourceDatatype;
+ protected IHyracksTaskContext ctx;
+ protected AdapterRuntimeManager runtimeManager;
+
+ public StreamBasedAdapter(ITupleParserFactory parserFactory, IAType sourceDatatype, IHyracksTaskContext ctx)
+ throws HyracksDataException {
+ this.tupleParser = parserFactory.createTupleParser(ctx);
+ this.sourceDatatype = sourceDatatype;
+ }
+
+ @Override
+ public void start(int partition, IFrameWriter writer) throws Exception {
+ InputStream in = getInputStream(partition);
+ tupleParser.parse(in, writer);
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/AlterFeedMessage.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/AlterFeedMessage.java
deleted file mode 100644
index c36dc03..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/AlterFeedMessage.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.external.feed.lifecycle;
-
-import java.util.Map;
-
-/**
- * A feed control message containing the altered values for
- * adapter configuration parameters. This message is dispatched
- * to all runtime instances of the feed's adapter.
- */
-public class AlterFeedMessage extends FeedMessage {
-
- private static final long serialVersionUID = 1L;
-
- private final Map<String, String> alteredConfParams;
-
- public AlterFeedMessage(Map<String, String> alteredConfParams) {
- super(MessageType.ALTER);
- this.alteredConfParams = alteredConfParams;
- }
-
- @Override
- public MessageType getMessageType() {
- return MessageType.ALTER;
- }
-
- public Map<String, String> getAlteredConfParams() {
- return alteredConfParams;
- }
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/FeedId.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/FeedId.java
deleted file mode 100644
index 94e679d..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/FeedId.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.external.feed.lifecycle;
-
-import java.io.Serializable;
-
-/**
- * A unique identifier for a feed (dataset).
- */
-public class FeedId implements Serializable {
-
- private static final long serialVersionUID = 1L;
-
- private final String dataverse;
- private final String dataset;
- private final int hashcode;
-
- public FeedId(String dataverse, String dataset) {
- this.dataset = dataset;
- this.dataverse = dataverse;
- this.hashcode = (dataverse + "." + dataset).hashCode();
- }
-
- public String getDataverse() {
- return dataverse;
- }
-
- public String getDataset() {
- return dataset;
- }
-
- @Override
- public boolean equals(Object o) {
- if (o == null || !(o instanceof FeedId)) {
- return false;
- }
- if (((FeedId) o).getDataset().equals(dataset) && ((FeedId) o).getDataverse().equals(dataverse)) {
- return true;
- }
- return false;
- }
-
- @Override
- public int hashCode() {
- return hashcode;
- }
-
- @Override
- public String toString() {
- return dataverse + "." + dataset;
- }
-
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/FeedManager.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/FeedManager.java
deleted file mode 100644
index 5b3ed35..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/FeedManager.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.external.feed.lifecycle;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-
-/**
- * Handle (de-)registration of feeds for delivery of control messages.
- */
-public class FeedManager implements IFeedManager {
-
- public static FeedManager INSTANCE = new FeedManager();
-
- private FeedManager() {
-
- }
-
- private Map<FeedId, Set<LinkedBlockingQueue<IFeedMessage>>> outGoingMsgQueueMap = new HashMap<FeedId, Set<LinkedBlockingQueue<IFeedMessage>>>();
-
- @Override
- public void deliverMessage(FeedId feedId, IFeedMessage feedMessage) throws AsterixException {
- Set<LinkedBlockingQueue<IFeedMessage>> operatorQueues = outGoingMsgQueueMap.get(feedId);
- try {
- if (operatorQueues != null) {
- for (LinkedBlockingQueue<IFeedMessage> queue : operatorQueues) {
- queue.put(feedMessage);
- }
- }
- } catch (Exception e) {
- throw new AsterixException(e);
- }
- }
-
- @Override
- public void registerFeedMsgQueue(FeedId feedId, LinkedBlockingQueue<IFeedMessage> queue) {
- Set<LinkedBlockingQueue<IFeedMessage>> feedQueues = outGoingMsgQueueMap.get(feedId);
- if (feedQueues == null) {
- feedQueues = new HashSet<LinkedBlockingQueue<IFeedMessage>>();
- }
- feedQueues.add(queue);
- outGoingMsgQueueMap.put(feedId, feedQueues);
- }
-
- @Override
- public void unregisterFeedMsgQueue(FeedId feedId, LinkedBlockingQueue<IFeedMessage> queue) {
- Set<LinkedBlockingQueue<IFeedMessage>> feedQueues = outGoingMsgQueueMap.get(feedId);
- if (feedQueues == null || !feedQueues.contains(queue)) {
- throw new IllegalArgumentException(" Unable to de-register feed message queue. Unknown feedId " + feedId);
- }
- feedQueues.remove(queue);
- }
-
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedManager.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedManager.java
deleted file mode 100644
index 2febbe4..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedManager.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.external.feed.lifecycle;
-
-import java.util.concurrent.LinkedBlockingQueue;
-
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-
-/**
- * Handle (de-)registration of feeds for delivery of control messages.
- */
-public interface IFeedManager {
-
- /**
- * Register an input message queue for a feed specified by feedId.
- * All messages sent to a feed are directed to the registered queue(s).
- *
- * @param feedId
- * an identifier for the feed dataset.
- * @param queue
- * an input message queue for receiving control messages.
- */
- public void registerFeedMsgQueue(FeedId feedId, LinkedBlockingQueue<IFeedMessage> queue);
-
- /**
- * Unregister an input message queue for a feed specified by feedId.
- * A feed prior to finishing should unregister all previously registered queue(s)
- * as it is no longer active and thus need not process any control messages.
- *
- * @param feedId
- * an identifier for the feed dataset.
- * @param queue
- * an input message queue for receiving control messages.
- */
- public void unregisterFeedMsgQueue(FeedId feedId, LinkedBlockingQueue<IFeedMessage> queue);
-
- /**
- * Deliver a message to a feed with a given feedId.
- *
- * @param feedId
- * identifier for the feed dataset.
- * @param feedMessage
- * control message that needs to be delivered.
- * @throws Exception
- */
- public void deliverMessage(FeedId feedId, IFeedMessage feedMessage) throws AsterixException;
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalFunction.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalFunction.java
new file mode 100755
index 0000000..3fd96ec
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalFunction.java
@@ -0,0 +1,75 @@
+package edu.uci.ics.asterix.external.library;
+
+import java.io.IOException;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.functions.IExternalFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public abstract class ExternalFunction implements IExternalFunction {
+
+ protected final IExternalFunctionInfo finfo;
+ protected final IFunctionFactory externalFunctionFactory;
+ protected final IExternalFunction externalFunction;
+ protected final ICopyEvaluatorFactory[] evaluatorFactories;
+ protected final IDataOutputProvider out;
+ protected final ArrayBackedValueStorage inputVal = new ArrayBackedValueStorage();
+ protected final ICopyEvaluator[] argumentEvaluators;
+ protected final JavaFunctionHelper functionHelper;
+
+ public ExternalFunction(IExternalFunctionInfo finfo, ICopyEvaluatorFactory args[],
+ IDataOutputProvider outputProvider) throws AlgebricksException {
+ this.finfo = finfo;
+ this.evaluatorFactories = args;
+ this.out = outputProvider;
+ argumentEvaluators = new ICopyEvaluator[args.length];
+ for (int i = 0; i < args.length; i++) {
+ argumentEvaluators[i] = args[i].createEvaluator(inputVal);
+ }
+ functionHelper = new JavaFunctionHelper(finfo, outputProvider);
+
+ String[] fnameComponents = finfo.getFunctionIdentifier().getName().split("#");
+ String functionLibary = fnameComponents[0];
+ String dataverse = finfo.getFunctionIdentifier().getNamespace();
+ ClassLoader libraryClassLoader = ExternalLibraryManager.getLibraryClassLoader(dataverse, functionLibary);
+ String classname = finfo.getFunctionBody().trim();
+ Class clazz;
+ try {
+ clazz = Class.forName(classname, true, libraryClassLoader);
+ externalFunctionFactory = (IFunctionFactory) clazz.newInstance();
+ externalFunction = externalFunctionFactory.getExternalFunction();
+ } catch (Exception e) {
+ throw new AlgebricksException(" Unable to load/instantiate class " + classname, e);
+ }
+ }
+
+ public static ISerializerDeserializer getSerDe(Object typeInfo) {
+ return AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(typeInfo);
+ }
+
+ public IExternalFunctionInfo getFinfo() {
+ return finfo;
+ }
+
+ public void setArguments(IFrameTupleReference tuple) throws AlgebricksException, IOException, AsterixException {
+ for (int i = 0; i < evaluatorFactories.length; i++) {
+ inputVal.reset();
+ argumentEvaluators[i].evaluate(tuple);
+ functionHelper.setArgument(i, inputVal.getByteArray());
+ }
+ }
+
+ @Override
+ public void deinitialize() {
+ externalFunction.deinitialize();
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalFunctionDescriptorProvider.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalFunctionDescriptorProvider.java
new file mode 100755
index 0000000..b5888a6
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalFunctionDescriptorProvider.java
@@ -0,0 +1,51 @@
+package edu.uci.ics.asterix.external.library;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.om.functions.IExternalFunctionInfo;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+
+public class ExternalFunctionDescriptorProvider {
+
+ public static IFunctionDescriptor getExternalFunctionDescriptor(IExternalFunctionInfo finfo) throws AsterixException {
+ switch (finfo.getKind()) {
+ case SCALAR:
+ return new ExternalScalarFunctionDescriptor(finfo);
+ case AGGREGATE:
+ case UNNEST:
+ throw new AsterixException("Unsupported function kind :" + finfo.getKind());
+ default:
+ break;
+ }
+ return null;
+ }
+
+}
+
+class ExternalScalarFunctionDescriptor extends AbstractScalarFunctionDynamicDescriptor implements IFunctionDescriptor {
+
+ private final IFunctionInfo finfo;
+ private ICopyEvaluatorFactory evaluatorFactory;
+ private ICopyEvaluatorFactory[] args;
+
+ @Override
+ public ICopyEvaluatorFactory createEvaluatorFactory(ICopyEvaluatorFactory[] args) throws AlgebricksException {
+ evaluatorFactory = new ExternalScalarFunctionEvaluatorFactory((IExternalFunctionInfo) finfo, args);
+ return evaluatorFactory;
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return finfo.getFunctionIdentifier();
+ }
+
+ public ExternalScalarFunctionDescriptor(IFunctionInfo finfo) {
+ this.finfo = finfo;
+ }
+
+}
+
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalFunctionProvider.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalFunctionProvider.java
new file mode 100755
index 0000000..db6c224
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalFunctionProvider.java
@@ -0,0 +1,65 @@
+package edu.uci.ics.asterix.external.library;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.om.functions.IExternalFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class ExternalFunctionProvider {
+
+ private static Map<IExternalFunctionInfo, ExternalScalarFunction> functionRepo = new HashMap<IExternalFunctionInfo, ExternalScalarFunction>();
+
+ public static IExternalFunction getExternalFunctionEvaluator(IExternalFunctionInfo finfo,
+ ICopyEvaluatorFactory args[], IDataOutputProvider outputProvider) throws AlgebricksException {
+ switch (finfo.getKind()) {
+ case SCALAR:
+ ExternalScalarFunction function = functionRepo.get(finfo);
+ function = new ExternalScalarFunction(finfo, args, outputProvider);
+ // functionRepo.put(finfo, function);
+ return function;
+ case AGGREGATE:
+ case UNNEST:
+ throw new IllegalArgumentException(" not supported function kind" + finfo.getKind());
+ default:
+ throw new IllegalArgumentException(" unknown function kind" + finfo.getKind());
+ }
+ }
+}
+
+class ExternalScalarFunction extends ExternalFunction implements IExternalScalarFunction, ICopyEvaluator {
+
+ public ExternalScalarFunction(IExternalFunctionInfo finfo, ICopyEvaluatorFactory args[],
+ IDataOutputProvider outputProvider) throws AlgebricksException {
+ super(finfo, args, outputProvider);
+ try {
+ initialize(functionHelper);
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ try {
+ setArguments(tuple);
+ evaluate(functionHelper);
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ public void evaluate(IFunctionHelper argumentProvider) throws Exception {
+ ((IExternalScalarFunction) externalFunction).evaluate(argumentProvider);
+ }
+
+ @Override
+ public void initialize(IFunctionHelper functionHelper) throws Exception {
+ ((IExternalScalarFunction) externalFunction).initialize(functionHelper);
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalLibraryManager.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalLibraryManager.java
new file mode 100755
index 0000000..520020d
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalLibraryManager.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.external.library;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class ExternalLibraryManager {
+
+ private static Map<String, ClassLoader> libraryClassLoaders = new HashMap<String, ClassLoader>();
+
+ public static void registerLibraryClassLoader(String dataverseName, String libraryName, ClassLoader classLoader) {
+ String key = dataverseName + "." + libraryName;
+ synchronized (libraryClassLoaders) {
+ if (libraryClassLoaders.get(dataverseName) != null) {
+ throw new IllegalStateException("library class loader already registered!");
+ }
+ libraryClassLoaders.put(key, classLoader);
+ }
+ }
+
+ public static void deregisterLibraryClassLoader(String dataverseName, String libraryName) {
+ String key = dataverseName + "." + libraryName;
+ synchronized (libraryClassLoaders) {
+ if (libraryClassLoaders.get(dataverseName) != null) {
+ libraryClassLoaders.remove(key);
+ }
+ }
+ }
+
+ public static ClassLoader getLibraryClassLoader(String dataverseName, String libraryName) {
+ String key = dataverseName + "." + libraryName;
+ synchronized (libraryClassLoaders) {
+ return libraryClassLoaders.get(key);
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalScalarFunctionEvaluatorFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalScalarFunctionEvaluatorFactory.java
new file mode 100755
index 0000000..a185000
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ExternalScalarFunctionEvaluatorFactory.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.external.library;
+
+import edu.uci.ics.asterix.om.functions.IExternalFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+
+public class ExternalScalarFunctionEvaluatorFactory implements ICopyEvaluatorFactory {
+
+ private final IExternalFunctionInfo finfo;
+ private final ICopyEvaluatorFactory[] args;
+
+ public ExternalScalarFunctionEvaluatorFactory(IExternalFunctionInfo finfo, ICopyEvaluatorFactory[] args)
+ throws AlgebricksException {
+ this.finfo = finfo;
+ this.args = args;
+ }
+
+ @Override
+ public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+ return (ExternalScalarFunction) ExternalFunctionProvider.getExternalFunctionEvaluator(finfo, args, output);
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IExternalFunction.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IExternalFunction.java
new file mode 100755
index 0000000..e667828
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IExternalFunction.java
@@ -0,0 +1,7 @@
+package edu.uci.ics.asterix.external.library;
+
+public interface IExternalFunction {
+
+ public void deinitialize();
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IExternalScalarFunction.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IExternalScalarFunction.java
new file mode 100755
index 0000000..09451d8
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IExternalScalarFunction.java
@@ -0,0 +1,9 @@
+package edu.uci.ics.asterix.external.library;
+
+public interface IExternalScalarFunction extends IExternalFunction {
+
+ public void initialize(IFunctionHelper functionHelper) throws Exception;
+
+ public void evaluate(IFunctionHelper functionHelper) throws Exception;
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IFunctionFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IFunctionFactory.java
new file mode 100755
index 0000000..f67957d
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IFunctionFactory.java
@@ -0,0 +1,7 @@
+package edu.uci.ics.asterix.external.library;
+
+public interface IFunctionFactory {
+
+ public IExternalFunction getExternalFunction();
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IFunctionHelper.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IFunctionHelper.java
new file mode 100755
index 0000000..0192e33
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IFunctionHelper.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.external.library;
+
+import java.io.IOException;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.external.library.java.IJObject;
+import edu.uci.ics.asterix.external.library.java.JTypeTag;
+
+public interface IFunctionHelper {
+
+ public IJObject getArgument(int index);
+
+ public IJObject getResultObject();
+
+ public void setResult(IJObject result) throws IOException, AsterixException;
+
+ public IJObject getObject(JTypeTag jtypeTag);
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IResultCollector.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IResultCollector.java
new file mode 100755
index 0000000..fee002e
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/IResultCollector.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.external.library;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.om.base.AOrderedList;
+import edu.uci.ics.asterix.om.base.ARecord;
+import edu.uci.ics.asterix.om.base.IAObject;
+
+public interface IResultCollector {
+
+ public void writeIntResult(int result) throws AsterixException;
+
+ public void writeFloatResult(float result) throws AsterixException;
+
+ public void writeDoubleResult(double result) throws AsterixException;
+
+ public void writeStringResult(String result) throws AsterixException;
+
+ public void writeRecordResult(ARecord result) throws AsterixException;
+
+ public void writeListResult(AOrderedList list) throws AsterixException;
+
+ public IAObject getComplexTypeResultHolder();
+
+ public DataOutput getDataOutput();
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/JTypeObjectFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/JTypeObjectFactory.java
new file mode 100644
index 0000000..ad56981
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/JTypeObjectFactory.java
@@ -0,0 +1,115 @@
+package edu.uci.ics.asterix.external.library;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.asterix.external.library.java.IJObject;
+import edu.uci.ics.asterix.external.library.java.JObjects.JBoolean;
+import edu.uci.ics.asterix.external.library.java.JObjects.JCircle;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDate;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDateTime;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDouble;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDuration;
+import edu.uci.ics.asterix.external.library.java.JObjects.JFloat;
+import edu.uci.ics.asterix.external.library.java.JObjects.JInt;
+import edu.uci.ics.asterix.external.library.java.JObjects.JInterval;
+import edu.uci.ics.asterix.external.library.java.JObjects.JLine;
+import edu.uci.ics.asterix.external.library.java.JObjects.JLong;
+import edu.uci.ics.asterix.external.library.java.JObjects.JOrderedList;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPoint;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPoint3D;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPolygon;
+import edu.uci.ics.asterix.external.library.java.JObjects.JRecord;
+import edu.uci.ics.asterix.external.library.java.JObjects.JRectangle;
+import edu.uci.ics.asterix.external.library.java.JObjects.JString;
+import edu.uci.ics.asterix.external.library.java.JObjects.JTime;
+import edu.uci.ics.asterix.external.library.java.JObjects.JUnorderedList;
+import edu.uci.ics.asterix.om.types.AOrderedListType;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.container.IObjectFactory;
+
+public class JTypeObjectFactory implements IObjectFactory<IJObject, IAType> {
+
+ @Override
+ public IJObject create(IAType type) {
+ IJObject retValue = null;
+ switch (type.getTypeTag()) {
+ case INT32:
+ retValue = new JInt(0);
+ break;
+ case STRING:
+ retValue = new JString("");
+ break;
+ case FLOAT:
+ retValue = new JFloat(0);
+ break;
+ case DOUBLE:
+ retValue = new JDouble(0);
+ break;
+ case BOOLEAN:
+ retValue = new JBoolean(false);
+ break;
+ case CIRCLE:
+ retValue = new JCircle(new JPoint(0, 0), 0);
+ break;
+ case POINT:
+ retValue = new JPoint(0, 0);
+ break;
+ case POINT3D:
+ retValue = new JPoint3D(0, 0, 0);
+ break;
+ case POLYGON:
+ retValue = new JPolygon(new ArrayList<JPoint>());
+ break;
+ case LINE:
+ retValue = new JLine(new JPoint(0, 0), new JPoint(0, 0));
+ break;
+ case RECTANGLE:
+ retValue = new JRectangle(new JPoint(0, 0), new JPoint(1, 1));
+ break;
+ case DATE:
+ retValue = new JDate(0);
+ break;
+ case DATETIME:
+ retValue = new JDateTime(0);
+ break;
+ case DURATION:
+ retValue = new JDuration(0, 0);
+ break;
+ case INTERVAL:
+ retValue = new JInterval(0, 0);
+ break;
+ case TIME:
+ retValue = new JTime(0);
+ break;
+ case INT64:
+ retValue = new JLong(0);
+ break;
+ case ORDEREDLIST:
+ AOrderedListType ot = (AOrderedListType) type;
+ IAType orderedItemType = ot.getItemType();
+ IJObject orderedItemObject = create(orderedItemType);
+ retValue = new JOrderedList(orderedItemObject);
+ break;
+ case UNORDEREDLIST:
+ AUnorderedListType ut = (AUnorderedListType) type;
+ IAType unorderedItemType = ut.getItemType();
+ IJObject unorderedItemObject = create(unorderedItemType);
+ retValue = new JUnorderedList(unorderedItemObject);
+ break;
+ case RECORD:
+ IAType[] fieldTypes = ((ARecordType) type).getFieldTypes();
+ IJObject[] fieldObjects = new IJObject[fieldTypes.length];
+ int index = 0;
+ for (IAType fieldType : fieldTypes) {
+ fieldObjects[index] = create(fieldType);
+ index++;
+ }
+ retValue = new JRecord((ARecordType) type, fieldObjects);
+
+ break;
+ }
+ return retValue;
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/JavaFunctionHelper.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/JavaFunctionHelper.java
new file mode 100644
index 0000000..8aed123
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/JavaFunctionHelper.java
@@ -0,0 +1,201 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.external.library;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import edu.uci.ics.asterix.builders.RecordBuilder;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.external.library.java.IJObject;
+import edu.uci.ics.asterix.external.library.java.JObjectUtil;
+import edu.uci.ics.asterix.external.library.java.JObjects.ByteArrayAccessibleDataInputStream;
+import edu.uci.ics.asterix.external.library.java.JObjects.ByteArrayAccessibleInputStream;
+import edu.uci.ics.asterix.external.library.java.JObjects.JRecord;
+import edu.uci.ics.asterix.external.library.java.JTypeTag;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ARecord;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.functions.IExternalFunctionInfo;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.container.IObjectPool;
+import edu.uci.ics.asterix.om.util.container.ListObjectPool;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+
+public class JavaFunctionHelper implements IFunctionHelper {
+
+ private final IExternalFunctionInfo finfo;
+ private final IDataOutputProvider outputProvider;
+ private IJObject[] arguments;
+ private IJObject resultHolder;
+ private IAObject innerResult;
+ private ISerializerDeserializer resultSerde;
+ private IObjectPool<IJObject, IAType> objectPool = new ListObjectPool<IJObject, IAType>(new JTypeObjectFactory());
+ byte[] buffer = new byte[32768];
+ ByteArrayAccessibleInputStream bis = new ByteArrayAccessibleInputStream(buffer, 0, buffer.length);
+ ByteArrayAccessibleDataInputStream dis = new ByteArrayAccessibleDataInputStream(bis);
+
+ public JavaFunctionHelper(IExternalFunctionInfo finfo, IDataOutputProvider outputProvider)
+ throws AlgebricksException {
+ this.finfo = finfo;
+ this.outputProvider = outputProvider;
+ List<IAType> params = finfo.getParamList();
+ arguments = new IJObject[params.size()];
+ int index = 0;
+ for (IAType param : params) {
+ this.arguments[index] = objectPool.allocate(param);
+ index++;
+ }
+ resultHolder = objectPool.allocate(finfo.getReturnType());
+ }
+
+ @Override
+ public IJObject getArgument(int index) {
+ return arguments[index];
+ }
+
+ @Override
+ public void setResult(IJObject result) throws IOException, AsterixException {
+ IAObject obj = result.getIAObject();
+ try {
+ outputProvider.getDataOutput().writeByte(obj.getType().getTypeTag().serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+
+ if (obj.getType().getTypeTag().equals(ATypeTag.RECORD)) {
+ ARecordType recType = (ARecordType) obj.getType();
+ if (recType.isOpen()) {
+ writeOpenRecord((JRecord) result, outputProvider.getDataOutput());
+ } else {
+ resultSerde = AqlSerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(recType);
+ resultSerde.serialize(obj, outputProvider.getDataOutput());
+ }
+ } else {
+ resultSerde = AqlSerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(obj.getType());
+ resultSerde.serialize(obj, outputProvider.getDataOutput());
+ }
+ reset();
+ }
+
+ private void writeOpenRecord(JRecord jRecord, DataOutput dataOutput) throws AsterixException, IOException {
+ ARecord aRecord = (ARecord) jRecord.getIAObject();
+ RecordBuilder recordBuilder = new RecordBuilder();
+ ARecordType recordType = aRecord.getType();
+ recordBuilder.reset(recordType);
+ ArrayBackedValueStorage fieldName = new ArrayBackedValueStorage();
+ ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
+ List<Boolean> openField = jRecord.getOpenField();
+
+ int fieldIndex = 0;
+ int closedFieldId = 0;
+ for (IJObject field : jRecord.getFields()) {
+ fieldValue.reset();
+ switch (field.getTypeTag()) {
+ case RECORD:
+ ARecordType recType = (ARecordType) field.getIAObject().getType();
+ if (recType.isOpen()) {
+ fieldValue.getDataOutput().writeByte(recType.getTypeTag().serialize());
+ writeOpenRecord((JRecord) field, fieldValue.getDataOutput());
+ } else {
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(
+ field.getIAObject().getType()).serialize(field.getIAObject(),
+ fieldValue.getDataOutput());
+ }
+ break;
+ default:
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(field.getIAObject().getType())
+ .serialize(field.getIAObject(), fieldValue.getDataOutput());
+ break;
+ }
+ if (openField.get(fieldIndex)) {
+ String fName = jRecord.getFieldNames().get(fieldIndex);
+ fieldName.reset();
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING).serialize(
+ new AString(fName), fieldName.getDataOutput());
+ recordBuilder.addField(fieldName, fieldValue);
+ } else {
+ recordBuilder.addField(closedFieldId, fieldValue);
+ closedFieldId++;
+ }
+ fieldIndex++;
+ }
+
+ recordBuilder.write(dataOutput, false);
+
+ }
+
+ private void reset() {
+ for (IJObject jObject : arguments) {
+ switch (jObject.getTypeTag()) {
+ case RECORD:
+ reset((JRecord) jObject);
+ break;
+ }
+ }
+ switch (resultHolder.getTypeTag()) {
+ case RECORD:
+ reset((JRecord) resultHolder);
+ break;
+ }
+ }
+
+ private void reset(JRecord jRecord) {
+ List<IJObject> fields = ((JRecord) jRecord).getFields();
+ for (IJObject field : fields) {
+ switch (field.getTypeTag()) {
+ case RECORD:
+ reset((JRecord) field);
+ break;
+ }
+ }
+ jRecord.close();
+ }
+
+ public void setArgument(int index, byte[] argument) throws IOException, AsterixException {
+ bis.setContent(argument, 1, argument.length);
+ IAType type = finfo.getParamList().get(index);
+ arguments[index] = JObjectUtil.getJType(type.getTypeTag(), type, dis, objectPool);
+ }
+
+ @Override
+ public IJObject getResultObject() {
+ return resultHolder;
+ }
+
+ @Override
+ public IJObject getObject(JTypeTag jtypeTag) {
+ IJObject retValue = null;
+ switch (jtypeTag) {
+ case INT:
+ retValue = objectPool.allocate(BuiltinType.AINT32);
+ break;
+ case STRING:
+ retValue = objectPool.allocate(BuiltinType.ASTRING);
+ break;
+ }
+ return retValue;
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ResultCollector.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ResultCollector.java
new file mode 100755
index 0000000..d53b044
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/ResultCollector.java
@@ -0,0 +1,147 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.external.library;
+
+import java.io.DataOutput;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AMutableDouble;
+import edu.uci.ics.asterix.om.base.AMutableFloat;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableOrderedList;
+import edu.uci.ics.asterix.om.base.AMutableRecord;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.AOrderedList;
+import edu.uci.ics.asterix.om.base.ARecord;
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.functions.IExternalFunctionInfo;
+import edu.uci.ics.asterix.om.types.AOrderedListType;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+
+public class ResultCollector implements IResultCollector {
+
+ private IAObject reusableResultObjectHolder;
+ private ByteBuffer reusableResultBinaryHolder;
+ private IDataOutputProvider outputProvider;
+ private IExternalFunctionInfo finfo;
+
+ public ResultCollector(IExternalFunctionInfo finfo, IDataOutputProvider outputProvider) {
+ this.finfo = finfo;
+ IAType returnType = finfo.getReturnType();
+ reusableResultObjectHolder = allocateResultObjectHolder(returnType);
+ reusableResultBinaryHolder = allocateResultBinaryHolder(returnType);
+ this.outputProvider = outputProvider;
+ }
+
+ private IAObject allocateResultObjectHolder(IAType type) {
+ switch (type.getTypeTag()) {
+ case INT32:
+ return new AMutableInt32(0);
+ case FLOAT:
+ return new AMutableFloat(0f);
+ case DOUBLE:
+ return new AMutableDouble(0);
+ case STRING:
+ return new AMutableString("");
+ case ORDEREDLIST:
+ return new AMutableOrderedList((AOrderedListType) type);
+ case RECORD:
+ IAType[] fieldType = ((ARecordType) type).getFieldTypes();
+ IAObject[] fieldObjects = new IAObject[fieldType.length];
+ for (int i = 0; i < fieldType.length; i++) {
+ fieldObjects[i] = allocateResultObjectHolder(fieldType[i]);
+ }
+ return new AMutableRecord((ARecordType) type, fieldObjects);
+ }
+ return null;
+ }
+
+ private ByteBuffer allocateResultBinaryHolder(IAType type) {
+ switch (type.getTypeTag()) {
+ case INT32:
+ return ByteBuffer.allocate(4);
+ case FLOAT:
+ return ByteBuffer.allocate(4);
+ case DOUBLE:
+ return ByteBuffer.allocate(8);
+ case STRING:
+ return ByteBuffer.allocate(32 * 1024);
+ case ORDEREDLIST:
+ return ByteBuffer.allocate(32 * 1024);
+ case RECORD:
+ return ByteBuffer.allocate(32 * 1024);
+ }
+ return null;
+ }
+
+ @Override
+ public void writeDoubleResult(double result) throws AsterixException {
+ ((AMutableDouble) reusableResultObjectHolder).setValue(result);
+ serializeResult(reusableResultObjectHolder);
+ }
+
+ @Override
+ public void writeFloatResult(float result) throws AsterixException {
+ ((AMutableDouble) reusableResultObjectHolder).setValue(result);
+ serializeResult(reusableResultObjectHolder);
+ }
+
+ @Override
+ public void writeIntResult(int result) throws AsterixException {
+ ((AMutableInt32) reusableResultObjectHolder).setValue(result);
+ serializeResult(reusableResultObjectHolder);
+ }
+
+ @Override
+ public void writeStringResult(String result) throws AsterixException {
+ ((AMutableString) reusableResultObjectHolder).setValue(result);
+ serializeResult(reusableResultObjectHolder);
+
+ }
+
+ @Override
+ public void writeRecordResult(ARecord result) throws AsterixException {
+ serializeResult(result);
+ }
+
+ @Override
+ public void writeListResult(AOrderedList list) throws AsterixException {
+ serializeResult(list);
+ }
+
+ public IAObject getComplexTypeResultHolder() {
+ return reusableResultObjectHolder;
+ }
+
+ private void serializeResult(IAObject object) throws AsterixException {
+ try {
+ AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(finfo.getReturnType()).serialize(
+ reusableResultObjectHolder, outputProvider.getDataOutput());
+ } catch (HyracksDataException hde) {
+ throw new AsterixException(hde);
+ }
+ }
+
+ @Override
+ public DataOutput getDataOutput() {
+ return outputProvider.getDataOutput();
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/RuntimeExternalFunctionUtil.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/RuntimeExternalFunctionUtil.java
new file mode 100755
index 0000000..0c6713d
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/RuntimeExternalFunctionUtil.java
@@ -0,0 +1,100 @@
+package edu.uci.ics.asterix.external.library;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableRecord;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.functions.IExternalFunctionInfo;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+
+public class RuntimeExternalFunctionUtil {
+
+ private static Map<String, ClassLoader> libraryClassLoaders = new HashMap<String, ClassLoader>();
+
+ public static void registerLibraryClassLoader(String dataverseName, String libraryName, ClassLoader classLoader) {
+ String key = dataverseName + "." + libraryName;
+ synchronized (libraryClassLoaders) {
+ if (libraryClassLoaders.get(dataverseName) != null) {
+ throw new IllegalStateException("library class loader already registered!");
+ }
+ libraryClassLoaders.put(key, classLoader);
+ }
+ }
+
+ public static ClassLoader getLibraryClassLoader(String dataverseName, String libraryName) {
+ String key = dataverseName + "." + libraryName;
+ synchronized (libraryClassLoaders) {
+ return libraryClassLoaders.get(key);
+ }
+ }
+
+ public static IFunctionDescriptor getFunctionDescriptor(IFunctionInfo finfo) {
+ switch (((IExternalFunctionInfo) finfo).getKind()) {
+ case SCALAR:
+ return getScalarFunctionDescriptor(finfo);
+ case AGGREGATE:
+ case UNNEST:
+ case STATEFUL:
+ throw new NotImplementedException("External " + finfo.getFunctionIdentifier().getName()
+ + " not supported");
+ }
+ return null;
+ }
+
+ private static AbstractScalarFunctionDynamicDescriptor getScalarFunctionDescriptor(IFunctionInfo finfo) {
+ return new ExternalScalarFunctionDescriptor(finfo);
+ }
+
+ public static ByteBuffer allocateArgumentBuffers(IAType type) {
+ switch (type.getTypeTag()) {
+ case INT32:
+ return ByteBuffer.allocate(4);
+ case STRING:
+ return ByteBuffer.allocate(32 * 1024);
+ default:
+ return ByteBuffer.allocate(32 * 1024);
+ }
+ }
+
+ public static IAObject allocateArgumentObjects(IAType type) {
+ switch (type.getTypeTag()) {
+ case INT32:
+ return new AMutableInt32(0);
+ case STRING:
+ return new AMutableString("");
+ default:
+ return null;
+ /*
+ ARecordType recordType = (ARecordType) type;
+ IAType[] fieldTypes = recordType.getFieldTypes();
+ IAObject[] fields = new IAObject[fieldTypes.length];
+ for (int i = 0; i < fields.length; i++) {
+ fields[i] = allocateArgumentObjects(fieldTypes[i]);
+ }
+ return new AMutableRecord((ARecordType) type, fields);
+ */
+ }
+ }
+
+ public static File getExternalLibraryDeployDir(String nodeId) {
+ String filePath = null;
+ if (nodeId != null) {
+ filePath = "edu.uci.ics.hyracks.control.nc.NodeControllerService" + "/" + nodeId + "/"
+ + "applications/asterix/expanded/external-lib/libraries";
+ } else {
+ filePath = "ClusterControllerService" + "/" + "applications/asterix/expanded/external-lib/libraries";
+
+ }
+ return new File(filePath);
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJObject.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJObject.java
new file mode 100644
index 0000000..1561c42
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJObject.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.asterix.external.library.java;
+
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+
+public interface IJObject {
+
+ public ATypeTag getTypeTag();
+
+ public IAObject getIAObject();
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJType.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJType.java
new file mode 100644
index 0000000..fcc35eb
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJType.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.asterix.external.library.java;
+
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+
+public interface IJType {
+
+ public ATypeTag getTypeTag();
+
+ public IAObject getIAObject();
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjectUtil.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjectUtil.java
new file mode 100644
index 0000000..52071af
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjectUtil.java
@@ -0,0 +1,399 @@
+package edu.uci.ics.asterix.external.library.java;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.SerializerDeserializerUtil;
+import edu.uci.ics.asterix.external.library.java.JObjects.ByteArrayAccessibleDataInputStream;
+import edu.uci.ics.asterix.external.library.java.JObjects.JBoolean;
+import edu.uci.ics.asterix.external.library.java.JObjects.JCircle;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDate;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDateTime;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDouble;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDuration;
+import edu.uci.ics.asterix.external.library.java.JObjects.JFloat;
+import edu.uci.ics.asterix.external.library.java.JObjects.JInt;
+import edu.uci.ics.asterix.external.library.java.JObjects.JInterval;
+import edu.uci.ics.asterix.external.library.java.JObjects.JLine;
+import edu.uci.ics.asterix.external.library.java.JObjects.JOrderedList;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPoint;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPoint3D;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPolygon;
+import edu.uci.ics.asterix.external.library.java.JObjects.JRecord;
+import edu.uci.ics.asterix.external.library.java.JObjects.JRectangle;
+import edu.uci.ics.asterix.external.library.java.JObjects.JString;
+import edu.uci.ics.asterix.external.library.java.JObjects.JTime;
+import edu.uci.ics.asterix.external.library.java.JObjects.JUnorderedList;
+import edu.uci.ics.asterix.om.pointables.base.IVisitablePointable;
+import edu.uci.ics.asterix.om.types.AOrderedListType;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
+import edu.uci.ics.asterix.om.util.container.IObjectPool;
+
+public class JObjectUtil {
+
+ public static IJObject getJType(ATypeTag typeTag, IAType type, ByteArrayAccessibleDataInputStream dis,
+ IObjectPool<IJObject, IAType> objectPool) throws IOException, AsterixException {
+ IJObject jObject;
+
+ switch (typeTag) {
+
+ case INT32: {
+ int v = dis.readInt();
+ jObject = objectPool.allocate(BuiltinType.AINT32);
+ ((JInt) jObject).setValue(v);
+ break;
+ }
+
+ case FLOAT: {
+ float v = dis.readFloat();
+ jObject = objectPool.allocate(BuiltinType.AFLOAT);
+ ((JFloat) jObject).setValue(v);
+ break;
+ }
+
+ case DOUBLE: {
+ double value = dis.readDouble();
+ jObject = objectPool.allocate(BuiltinType.ADOUBLE);
+ ((JDouble) jObject).setValue(value);
+ break;
+ }
+
+ case STRING: {
+ String v = dis.readUTF();
+ jObject = objectPool.allocate(BuiltinType.ASTRING);
+ ((JString) jObject).setValue(v);
+ break;
+ }
+
+ case BOOLEAN:
+ jObject = objectPool.allocate(BuiltinType.ABOOLEAN);
+ ((JBoolean) jObject).setValue(dis.readBoolean());
+ break;
+
+ case DATE: {
+ int d = dis.readInt();
+ jObject = objectPool.allocate(BuiltinType.ADATE);
+ ((JDate) jObject).setValue(d);
+ break;
+ }
+
+ case DATETIME: {
+ jObject = objectPool.allocate(BuiltinType.ADATETIME);
+ long value = dis.readLong();
+ ((JDateTime) jObject).setValue(value);
+ break;
+ }
+
+ case DURATION: {
+ jObject = objectPool.allocate(BuiltinType.ADURATION);
+ int months = dis.readInt();
+ long msecs = dis.readLong();
+ ((JDuration) jObject).setValue(months, msecs);
+ break;
+ }
+
+ case TIME: {
+ jObject = objectPool.allocate(BuiltinType.ATIME);
+ int time = dis.readInt();
+ ((JTime) jObject).setValue(time);
+ break;
+ }
+
+ case INTERVAL: {
+ jObject = objectPool.allocate(BuiltinType.AINTERVAL);
+ long start = dis.readLong();
+ long end = dis.readLong();
+ byte intervalType = dis.readByte();
+ ((JInterval) jObject).setValue(start, end, intervalType);
+ break;
+ }
+
+ case CIRCLE: {
+ jObject = objectPool.allocate(BuiltinType.ACIRCLE);
+ double x = dis.readDouble();
+ double y = dis.readDouble();
+ double radius = dis.readDouble();
+ JPoint jpoint = (JPoint) objectPool.allocate(BuiltinType.APOINT);
+ jpoint.setValue(x, y);
+ ((JCircle) jObject).setValue(jpoint, radius);
+ break;
+ }
+
+ case POINT: {
+ jObject = objectPool.allocate(BuiltinType.APOINT);
+ double x = dis.readDouble();
+ double y = dis.readDouble();
+ ((JPoint) jObject).setValue(x, y);
+ break;
+ }
+
+ case POINT3D: {
+ jObject = objectPool.allocate(BuiltinType.APOINT3D);
+ double x = dis.readDouble();
+ double y = dis.readDouble();
+ double z = dis.readDouble();
+ ((JPoint3D) jObject).setValue(x, y, z);
+ break;
+ }
+
+ case LINE: {
+ jObject = objectPool.allocate(BuiltinType.ALINE);
+ double x1 = dis.readDouble();
+ double y1 = dis.readDouble();
+ double x2 = dis.readDouble();
+ double y2 = dis.readDouble();
+ JPoint jpoint1 = (JPoint) objectPool.allocate(BuiltinType.APOINT);
+ jpoint1.setValue(x1, y1);
+ JPoint jpoint2 = (JPoint) objectPool.allocate(BuiltinType.APOINT);
+ jpoint2.setValue(x2, y2);
+ ((JLine) jObject).setValue(jpoint1, jpoint2);
+ break;
+ }
+
+ case POLYGON: {
+ jObject = objectPool.allocate(BuiltinType.APOLYGON);
+ short numberOfPoints = dis.readShort();
+ List<JPoint> points = new ArrayList<JPoint>();
+ for (int i = 0; i < numberOfPoints; i++) {
+ JPoint p1 = (JPoint) objectPool.allocate(BuiltinType.APOINT);
+ p1.setValue(dis.readDouble(), dis.readDouble());
+ points.add(p1);
+ }
+ ((JPolygon) jObject).setValue(points);
+ break;
+ }
+
+ case RECTANGLE: {
+ jObject = objectPool.allocate(BuiltinType.ARECTANGLE);
+ double x1 = dis.readDouble();
+ double y1 = dis.readDouble();
+ double x2 = dis.readDouble();
+ double y2 = dis.readDouble();
+ JPoint jpoint1 = (JPoint) objectPool.allocate(BuiltinType.APOINT);
+ jpoint1.setValue(x1, y1);
+ JPoint jpoint2 = (JPoint) objectPool.allocate(BuiltinType.APOINT);
+ jpoint2.setValue(x2, y2);
+ ((JRectangle) jObject).setValue(jpoint1, jpoint2);
+ break;
+ }
+
+ case UNORDEREDLIST: {
+ AUnorderedListType listType = (AUnorderedListType) type;
+ IAType elementType = listType.getItemType();
+ jObject = objectPool.allocate(listType);
+
+ boolean fixedSize = false;
+ ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(dis.readByte());
+ switch (tag) {
+ case STRING:
+ case RECORD:
+ case ORDEREDLIST:
+ case UNORDEREDLIST:
+ case ANY:
+ fixedSize = false;
+ break;
+ default:
+ fixedSize = true;
+ break;
+ }
+ dis.readInt(); // list size
+ int numberOfitems;
+ numberOfitems = dis.readInt();
+ if (numberOfitems > 0) {
+ if (!fixedSize) {
+ for (int i = 0; i < numberOfitems; i++)
+ dis.readInt();
+ }
+ for (int i = 0; i < numberOfitems; i++) {
+ IJObject v = (IJObject) getJType(elementType.getTypeTag(), elementType, dis, objectPool);
+ ((JUnorderedList) jObject).add(v);
+ }
+ }
+
+ break;
+ }
+ case ORDEREDLIST: {
+ AOrderedListType listType = (AOrderedListType) type;
+ IAType elementType = listType.getItemType();
+ jObject = objectPool.allocate(listType);
+ boolean fixedSize = false;
+ ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(dis.readByte());
+ switch (tag) {
+ case STRING:
+ case RECORD:
+ case ORDEREDLIST:
+ case UNORDEREDLIST:
+ case ANY:
+ fixedSize = false;
+ break;
+ default:
+ fixedSize = true;
+ break;
+ }
+
+ dis.readInt(); // list size
+ int numberOfitems;
+ numberOfitems = dis.readInt();
+ if (numberOfitems > 0) {
+ if (!fixedSize) {
+ for (int i = 0; i < numberOfitems; i++)
+ dis.readInt();
+ }
+ for (int i = 0; i < numberOfitems; i++) {
+ IJObject v = (IJObject) getJType(elementType.getTypeTag(), elementType, dis, objectPool);
+ ((JOrderedList) jObject).add(v);
+ }
+ }
+
+ break;
+ }
+ case RECORD:
+ ARecordType recordType = (ARecordType) type;
+ int numberOfSchemaFields = recordType.getFieldTypes().length;
+ byte[] recordBits = dis.getInputStream().getArray();
+ boolean isExpanded = false;
+ int s = dis.getInputStream().getPosition();
+ int recordOffset = s;
+ int openPartOffset = 0;
+ int offsetArrayOffset = 0;
+ int[] fieldOffsets = new int[numberOfSchemaFields];
+ IJObject[] closedFields = new IJObject[numberOfSchemaFields];
+
+ if (recordType == null) {
+ openPartOffset = s + AInt32SerializerDeserializer.getInt(recordBits, s + 6);
+ s += 8;
+ isExpanded = true;
+ } else {
+ dis.skip(4); // reading length is not required.
+ if (recordType.isOpen()) {
+ isExpanded = dis.readBoolean();
+ if (isExpanded) {
+ openPartOffset = s + dis.readInt(); // AInt32SerializerDeserializer.getInt(recordBits, s + 6);
+ } else {
+ // do nothing s += 6;
+ }
+ } else {
+ // do nothing s += 5;
+ }
+ }
+
+ if (numberOfSchemaFields > 0) {
+ int numOfSchemaFields = dis.readInt(); //s += 4;
+ int nullBitMapOffset = 0;
+ boolean hasNullableFields = NonTaggedFormatUtil.hasNullableField(recordType);
+ if (hasNullableFields) {
+ nullBitMapOffset = dis.getInputStream().getPosition();//s
+ offsetArrayOffset = dis.getInputStream().getPosition() //s
+ + (numberOfSchemaFields % 8 == 0 ? numberOfSchemaFields / 8
+ : numberOfSchemaFields / 8 + 1);
+ } else {
+ offsetArrayOffset = dis.getInputStream().getPosition();
+ }
+ for (int i = 0; i < numberOfSchemaFields; i++) {
+ fieldOffsets[i] = dis.readInt(); // AInt32SerializerDeserializer.getInt(recordBits, offsetArrayOffset) + recordOffset;
+ // offsetArrayOffset += 4;
+ }
+ for (int fieldNumber = 0; fieldNumber < numberOfSchemaFields; fieldNumber++) {
+ if (hasNullableFields) {
+ byte b1 = recordBits[nullBitMapOffset + fieldNumber / 8];
+ int p = 1 << (7 - (fieldNumber % 8));
+ if ((b1 & p) == 0) {
+ // set null value (including type tag inside)
+ //fieldValues.add(nullReference);
+ continue;
+ }
+ }
+ IAType[] fieldTypes = recordType.getFieldTypes();
+ ATypeTag fieldValueTypeTag = null;
+
+ IAType fieldType = fieldTypes[fieldNumber];
+ if (fieldTypes[fieldNumber].getTypeTag() == ATypeTag.UNION) {
+ if (NonTaggedFormatUtil.isOptionalField((AUnionType) fieldTypes[fieldNumber])) {
+ fieldType = ((AUnionType) fieldTypes[fieldNumber]).getUnionList().get(
+ NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST);
+ fieldValueTypeTag = fieldType.getTypeTag();
+ // fieldValueLength = NonTaggedFormatUtil.getFieldValueLength(recordBits,
+ // fieldOffsets[fieldNumber], typeTag, false);
+ }
+ } else {
+ fieldValueTypeTag = fieldTypes[fieldNumber].getTypeTag();
+ }
+ closedFields[fieldNumber] = getJType(fieldValueTypeTag, fieldType, dis, objectPool);
+ }
+ }
+ if (isExpanded) {
+ int numberOfOpenFields = dis.readInt();
+ String[] fieldNames = new String[numberOfOpenFields];
+ IAType[] fieldTypes = new IAType[numberOfOpenFields];
+ IJObject[] openFields = new IJObject[numberOfOpenFields];
+ for (int i = 0; i < numberOfOpenFields; i++) {
+ dis.readInt();
+ dis.readInt();
+ }
+ for (int i = 0; i < numberOfOpenFields; i++) {
+ fieldNames[i] = AStringSerializerDeserializer.INSTANCE.deserialize(dis).getStringValue();
+ ATypeTag openFieldTypeTag = SerializerDeserializerUtil.deserializeTag(dis);
+ openFields[i] = getJType(openFieldTypeTag, null, dis, objectPool);
+ fieldTypes[i] = openFields[i].getIAObject().getType();
+ }
+ ARecordType openPartRecType = new ARecordType(null, fieldNames, fieldTypes, true);
+ if (numberOfSchemaFields > 0) {
+ ARecordType mergedRecordType = mergeRecordTypes(recordType, openPartRecType);
+ IJObject[] mergedFields = mergeFields(closedFields, openFields);
+ jObject = objectPool.allocate(recordType);
+ return new JRecord(mergedRecordType, mergedFields);
+ } else {
+ return new JRecord(recordType, openFields);
+ }
+ } else {
+ return new JRecord(recordType, closedFields);
+ }
+
+ default:
+ throw new IllegalStateException("Argument type: " + typeTag);
+ }
+ return jObject;
+ }
+
+ private static IJObject[] mergeFields(IJObject[] closedFields, IJObject[] openFields) {
+ IJObject[] fields = new IJObject[closedFields.length + openFields.length];
+ int i = 0;
+ for (; i < closedFields.length; i++) {
+ fields[i] = closedFields[i];
+ }
+ for (int j = 0; j < openFields.length; j++) {
+ fields[closedFields.length + j] = openFields[j];
+ }
+ return fields;
+ }
+
+ private static ARecordType mergeRecordTypes(ARecordType recType1, ARecordType recType2) throws AsterixException {
+
+ String[] fieldNames = new String[recType1.getFieldNames().length + recType2.getFieldNames().length];
+ IAType[] fieldTypes = new IAType[recType1.getFieldTypes().length + recType2.getFieldTypes().length];
+
+ int i = 0;
+ for (; i < recType1.getFieldNames().length; i++) {
+ fieldNames[i] = recType1.getFieldNames()[i];
+ fieldTypes[i] = recType1.getFieldTypes()[i];
+ }
+
+ for (int j = 0; j < recType2.getFieldNames().length; i++, j++) {
+ fieldNames[i] = recType2.getFieldNames()[j];
+ fieldTypes[i] = recType2.getFieldTypes()[j];
+ }
+ return new ARecordType(null, fieldNames, fieldTypes, true);
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjects.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjects.java
new file mode 100644
index 0000000..2fae902
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjects.java
@@ -0,0 +1,875 @@
+package edu.uci.ics.asterix.external.library.java;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.om.base.ABoolean;
+import edu.uci.ics.asterix.om.base.AMutableCircle;
+import edu.uci.ics.asterix.om.base.AMutableDate;
+import edu.uci.ics.asterix.om.base.AMutableDateTime;
+import edu.uci.ics.asterix.om.base.AMutableDouble;
+import edu.uci.ics.asterix.om.base.AMutableDuration;
+import edu.uci.ics.asterix.om.base.AMutableFloat;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt64;
+import edu.uci.ics.asterix.om.base.AMutableInterval;
+import edu.uci.ics.asterix.om.base.AMutableLine;
+import edu.uci.ics.asterix.om.base.AMutableOrderedList;
+import edu.uci.ics.asterix.om.base.AMutablePoint;
+import edu.uci.ics.asterix.om.base.AMutablePoint3D;
+import edu.uci.ics.asterix.om.base.AMutablePolygon;
+import edu.uci.ics.asterix.om.base.AMutableRecord;
+import edu.uci.ics.asterix.om.base.AMutableRectangle;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.AMutableTime;
+import edu.uci.ics.asterix.om.base.AMutableUnorderedList;
+import edu.uci.ics.asterix.om.base.APoint;
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.types.AOrderedListType;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+
+public class JObjects {
+
+ public static abstract class JObject implements IJObject {
+
+ protected IAObject value;
+ protected byte[] bytes;
+
+ protected JObject(IAObject value) {
+ this.value = value;
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return value.getType().getTypeTag();
+ }
+
+ @Override
+ public IAObject getIAObject() {
+ return value;
+ }
+ }
+
+ public static final class JInt implements IJObject {
+
+ private AMutableInt32 value;
+
+ public JInt(int value) {
+ this.value = new AMutableInt32(value);
+ }
+
+ public void setValue(int v) {
+ if (value == null) {
+ value = new AMutableInt32(v);
+ } else {
+ ((AMutableInt32) value).setValue(v);
+ }
+ }
+
+ public void setValue(AMutableInt32 v) {
+ value = v;
+ }
+
+ public int getValue() {
+ return ((AMutableInt32) value).getIntegerValue().intValue();
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return BuiltinType.AINT32.getTypeTag();
+ }
+
+ @Override
+ public IAObject getIAObject() {
+ return value;
+ }
+
+ }
+
+ public static final class JBoolean implements IJObject {
+
+ private boolean value;
+
+ public JBoolean(boolean value) {
+ this.value = value;
+ }
+
+ public void setValue(boolean value) {
+ this.value = value;
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.BOOLEAN;
+ }
+
+ @Override
+ public IAObject getIAObject() {
+ return value ? ABoolean.TRUE : ABoolean.FALSE;
+ }
+
+ }
+
+ public static final class JLong extends JObject {
+
+ public JLong(long v) {
+ super(new AMutableInt64(v));
+ }
+
+ public void setValue(long v) {
+ ((AMutableInt64) value).setValue(v);
+ }
+
+ public long getValue() {
+ return ((AMutableInt64) value).getLongValue();
+ }
+
+ }
+
+ public static final class JDouble extends JObject {
+
+ public JDouble(double v) {
+ super(new AMutableDouble(v));
+ }
+
+ public void setValue(double v) {
+ ((AMutableDouble) value).setValue(v);
+ }
+
+ public double getValue() {
+ return ((AMutableDouble) value).getDoubleValue();
+ }
+
+ }
+
+ public static final class JString extends JObject {
+
+ public JString(String v) {
+ super(new AMutableString(v));
+ }
+
+ public void setValue(String v) {
+ ((AMutableString) value).setValue(v);
+ }
+
+ public String getValue() {
+ return ((AMutableString) value).getStringValue();
+ }
+
+ }
+
+ public static final class JFloat implements IJObject {
+
+ private AMutableFloat value;
+
+ public JFloat(float v) {
+ value = new AMutableFloat(v);
+ }
+
+ public void setValue(float v) {
+ ((AMutableFloat) value).setValue(v);
+ }
+
+ public float getValue() {
+ return ((AMutableFloat) value).getFloatValue();
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return BuiltinType.AFLOAT.getTypeTag();
+ }
+
+ @Override
+ public IAObject getIAObject() {
+ return value;
+ }
+
+ }
+
+ public static final class JPoint extends JObject {
+
+ public JPoint(double x, double y) {
+ super(new AMutablePoint(x, y));
+ }
+
+ public void setValue(double x, double y) {
+ ((AMutablePoint) value).setValue(x, y);
+ }
+
+ public double getXValue() {
+ return ((AMutablePoint) value).getX();
+ }
+
+ public double getYValue() {
+ return ((AMutablePoint) value).getY();
+ }
+
+ public IAObject getValue() {
+ return value;
+ }
+
+ @Override
+ public String toString() {
+ return value.toString();
+ }
+ }
+
+ public static final class JRectangle implements IJObject {
+
+ private AMutableRectangle rect;
+
+ public JRectangle(JPoint p1, JPoint p2) {
+ rect = new AMutableRectangle((APoint) p1.getValue(), (APoint) p2.getValue());
+ }
+
+ public void setValue(JPoint p1, JPoint p2) {
+ this.rect.setValue((APoint) p1.getValue(), (APoint) p2.getValue());
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.RECTANGLE;
+ }
+
+ @Override
+ public IAObject getIAObject() {
+ return rect;
+ }
+
+ @Override
+ public String toString() {
+ return rect.toString();
+ }
+
+ }
+
+ public static final class JTime implements IJObject {
+
+ private AMutableTime time;
+
+ public JTime(int timeInMillsec) {
+ time = new AMutableTime(timeInMillsec);
+ }
+
+ public void setValue(int timeInMillsec) {
+ time.setValue(timeInMillsec);
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.TIME;
+ }
+
+ @Override
+ public IAObject getIAObject() {
+ return time;
+ }
+
+ @Override
+ public String toString() {
+ return time.toString();
+ }
+
+ }
+
+ public static final class JInterval implements IJObject {
+
+ private AMutableInterval interval;
+
+ public JInterval(long intervalStart, long intervalEnd) {
+ interval = new AMutableInterval(intervalStart, intervalEnd, (byte) 0);
+ }
+
+ public void setValue(long intervalStart, long intervalEnd, byte typetag) {
+ interval.setValue(intervalStart, intervalEnd, typetag);
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.INTERVAL;
+ }
+
+ @Override
+ public IAObject getIAObject() {
+ return interval;
+ }
+
+ @Override
+ public String toString() {
+ return interval.toString();
+ }
+
+ public long getIntervalStart() {
+ return interval.getIntervalStart();
+ }
+
+ public long getIntervalEnd() {
+ return interval.getIntervalEnd();
+ }
+
+ public short getIntervalType() {
+ return interval.getIntervalType();
+ }
+
+ }
+
+ public static final class JDate implements IJObject {
+
+ private AMutableDate date;
+
+ public JDate(int chrononTimeInDays) {
+ date = new AMutableDate(chrononTimeInDays);
+ }
+
+ public void setValue(int chrononTimeInDays) {
+ date.setValue(chrononTimeInDays);
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.DATE;
+ }
+
+ @Override
+ public IAObject getIAObject() {
+ return date;
+ }
+
+ @Override
+ public String toString() {
+ return date.toString();
+ }
+
+ }
+
+ public static final class JDateTime implements IJObject {
+
+ private AMutableDateTime dateTime;
+
+ public JDateTime(long chrononTime) {
+ dateTime = new AMutableDateTime(chrononTime);
+ }
+
+ public void setValue(long chrononTime) {
+ dateTime.setValue(chrononTime);
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.DATETIME;
+ }
+
+ @Override
+ public IAObject getIAObject() {
+ return dateTime;
+ }
+
+ @Override
+ public String toString() {
+ return dateTime.toString();
+ }
+
+ }
+
+ public static final class JDuration implements IJObject {
+
+ private AMutableDuration duration;
+
+ public JDuration(int months, long milliseconds) {
+ duration = new AMutableDuration(months, milliseconds);
+ }
+
+ public void setValue(int months, long milliseconds) {
+ duration.setValue(months, milliseconds);
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.DURATION;
+ }
+
+ @Override
+ public IAObject getIAObject() {
+ return duration;
+ }
+
+ @Override
+ public String toString() {
+ return duration.toString();
+ }
+
+ }
+
+ public static final class JPolygon implements IJObject {
+
+ private AMutablePolygon polygon;
+ private List<JPoint> points;
+
+ public JPolygon(List<JPoint> points) {
+ this.points = points;
+ }
+
+ public void setValue(List<JPoint> points) {
+ this.points = points;
+ polygon = null;
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.POLYGON;
+ }
+
+ @Override
+ public IAObject getIAObject() {
+ if (polygon == null) {
+ APoint[] pts = new APoint[points.size()];
+ int index = 0;
+ for (JPoint p : points) {
+ pts[index++] = (APoint) p.getIAObject();
+ }
+ polygon = new AMutablePolygon(pts);
+ }
+ return polygon;
+ }
+
+ @Override
+ public String toString() {
+ return getIAObject().toString();
+ }
+
+ }
+
+ public static final class JCircle implements IJObject {
+
+ private AMutableCircle circle;
+
+ public JCircle(JPoint center, double radius) {
+ circle = new AMutableCircle((APoint) center.getIAObject(), radius);
+ }
+
+ public void setValue(JPoint center, double radius) {
+ circle.setValue((APoint) center.getIAObject(), radius);
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.CIRCLE;
+ }
+
+ @Override
+ public IAObject getIAObject() {
+ return circle;
+ }
+
+ @Override
+ public String toString() {
+ return circle.toString();
+ }
+
+ }
+
+ public static final class JLine implements IJObject {
+
+ private AMutableLine line;
+
+ public JLine(JPoint p1, JPoint p2) {
+ line = new AMutableLine((APoint) p1.getIAObject(), (APoint) p2.getIAObject());
+ }
+
+ public void setValue(JPoint p1, JPoint p2) {
+ line.setValue((APoint) p1.getIAObject(), (APoint) p2.getIAObject());
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.LINE;
+ }
+
+ @Override
+ public IAObject getIAObject() {
+ return line;
+ }
+
+ @Override
+ public String toString() {
+ return line.toString();
+ }
+
+ }
+
+ public static final class JPoint3D implements IJObject {
+
+ private AMutablePoint3D value;
+
+ public JPoint3D(double x, double y, double z) {
+ value = new AMutablePoint3D(x, y, z);
+ }
+
+ public void setValue(double x, double y, double z) {
+ value.setValue(x, y, z);
+ }
+
+ public double getXValue() {
+ return ((AMutablePoint3D) value).getX();
+ }
+
+ public double getYValue() {
+ return ((AMutablePoint3D) value).getY();
+ }
+
+ public double getZValue() {
+ return ((AMutablePoint3D) value).getZ();
+ }
+
+ public IAObject getValue() {
+ return value;
+ }
+
+ @Override
+ public String toString() {
+ return value.toString();
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.POINT3D;
+ }
+
+ @Override
+ public IAObject getIAObject() {
+ return value;
+ }
+ }
+
+ public static final class JOrderedList implements IJObject {
+
+ private AOrderedListType listType;
+ private List<IJObject> jObjects;
+
+ public JOrderedList(IJObject jObject) {
+ this.listType = new AOrderedListType(jObject.getIAObject().getType(), null);
+ this.jObjects = new ArrayList<IJObject>();
+ }
+
+ public void add(IJObject jObject) {
+ jObjects.add(jObject);
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.ORDEREDLIST;
+ }
+
+ @Override
+ public IAObject getIAObject() {
+ AMutableOrderedList v = new AMutableOrderedList(listType);
+ for (IJObject jObj : jObjects) {
+ v.add(jObj.getIAObject());
+ }
+ return v;
+ }
+
+ public AOrderedListType getListType() {
+ return listType;
+ }
+
+ public void addAll(Collection<IJObject> jObjectCollection) {
+ jObjects.addAll(jObjectCollection);
+ }
+
+ public void clear() {
+ jObjects.clear();
+ }
+
+ public IJObject getElement(int index) {
+ return jObjects.get(index);
+ }
+
+ public int size() {
+ return jObjects.size();
+ }
+
+ }
+
+ public static final class JUnorderedList implements IJObject {
+
+ private AUnorderedListType listType;
+ private List<IJObject> jObjects;
+
+ public JUnorderedList(IJObject jObject) {
+ this.listType = new AUnorderedListType(jObject.getIAObject().getType(), null);
+ this.jObjects = new ArrayList<IJObject>();
+ }
+
+ public void add(IJObject jObject) {
+ jObjects.add(jObject);
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.UNORDEREDLIST;
+ }
+
+ @Override
+ public IAObject getIAObject() {
+ AMutableUnorderedList v = new AMutableUnorderedList(listType);
+ for (IJObject jObj : jObjects) {
+ v.add(jObj.getIAObject());
+ }
+ return v;
+ }
+
+ public AUnorderedListType getListType() {
+ return listType;
+ }
+
+ public boolean isEmpty() {
+ return jObjects.isEmpty();
+ }
+
+ public void addAll(Collection<IJObject> jObjectCollection) {
+ jObjects.addAll(jObjectCollection);
+ }
+
+ public void clear() {
+ jObjects.clear();
+ }
+
+ public IJObject getElement(int index) {
+ return jObjects.get(index);
+ }
+
+ public int size() {
+ return jObjects.size();
+ }
+
+ }
+
+ public static final class JRecord implements IJObject {
+
+ private AMutableRecord value;
+ private ARecordType recordType;
+ private List<IJObject> fields;
+ private List<String> fieldNames;
+ private List<IAType> fieldTypes;
+ private int numFieldsAdded = 0;
+ private List<Boolean> openField;
+
+ public JRecord(ARecordType recordType) {
+ this.recordType = recordType;
+ this.fields = new ArrayList<IJObject>();
+ initFieldInfo();
+ }
+
+ public JRecord(ARecordType recordType, IJObject[] fields) {
+ this.recordType = recordType;
+ this.fields = new ArrayList<IJObject>();
+ for (IJObject jObject : fields) {
+ this.fields.add(jObject);
+ }
+ initFieldInfo();
+ }
+
+ public JRecord(String[] fieldNames, IJObject[] fields) throws AsterixException {
+ this.recordType = getARecordType(fieldNames, fields);
+ this.fields = new ArrayList<IJObject>();
+ for (IJObject jObject : fields) {
+ this.fields.add(jObject);
+ }
+ initFieldInfo();
+ }
+
+ private ARecordType getARecordType(String[] fieldNames, IJObject[] fields) throws AsterixException {
+ IAType[] fieldTypes = new IAType[fields.length];
+ int index = 0;
+ for (IJObject jObj : fields) {
+ fieldTypes[index++] = jObj.getIAObject().getType();
+ }
+ ARecordType recordType = new ARecordType(null, fieldNames, fieldTypes, false);
+ return recordType;
+ }
+
+ private void initFieldInfo() {
+ this.openField = new ArrayList<Boolean>();
+ fieldNames = new ArrayList<String>();
+ for (String name : recordType.getFieldNames()) {
+ fieldNames.add(name);
+ openField.add(false);
+ }
+ fieldTypes = new ArrayList<IAType>();
+ for (IAType type : recordType.getFieldTypes()) {
+ fieldTypes.add(type);
+ }
+
+ }
+
+ private IAObject[] getIAObjectArray(List<IJObject> fields) {
+ IAObject[] retValue = new IAObject[fields.size()];
+ int index = 0;
+ for (IJObject jObject : fields) {
+ retValue[index++] = getIAObject(jObject);
+ }
+ return retValue;
+ }
+
+ private IAObject getIAObject(IJObject jObject) {
+ IAObject retVal = null;
+ switch (jObject.getTypeTag()) {
+ case RECORD:
+ ARecordType recType = ((JRecord) jObject).getRecordType();
+ IAObject[] fields = new IAObject[((JRecord) jObject).getFields().size()];
+ int index = 0;
+ for (IJObject field : ((JRecord) jObject).getFields()) {
+ fields[index++] = getIAObject(field);
+ }
+ retVal = new AMutableRecord(recType, fields);
+ default:
+ retVal = jObject.getIAObject();
+ break;
+ }
+ return retVal;
+ }
+
+ public void addField(String fieldName, IJObject fieldValue) {
+ int pos = getFieldPosByName(fieldName);
+ if (pos >= 0) {
+ throw new IllegalArgumentException("field already defined");
+ }
+ numFieldsAdded++;
+ fields.add(fieldValue);
+ fieldNames.add(fieldName);
+ fieldTypes.add(fieldValue.getIAObject().getType());
+ openField.add(true);
+ }
+
+ public IJObject getValueByName(String fieldName) throws AsterixException, IOException {
+ int fieldPos = getFieldPosByName(fieldName);
+ if (fieldPos < 0) {
+ throw new AsterixException("unknown field: " + fieldName);
+ }
+ return fields.get(fieldPos);
+ }
+
+ public void setValueAtPos(int pos, IJObject jtype) {
+ fields.set(pos, jtype);
+ }
+
+ public void setValue(AMutableRecord mutableRecord) {
+ this.value = mutableRecord;
+ this.recordType = mutableRecord.getType();
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return recordType.getTypeTag();
+ }
+
+ public void setField(String fieldName, IJObject fieldValue) {
+ int pos = getFieldPosByName(fieldName);
+ fields.set(pos, fieldValue);
+ if (value != null) {
+ value.setValueAtPos(pos, fieldValue.getIAObject());
+ }
+ }
+
+ private int getFieldPosByName(String fieldName) {
+ int index = 0;
+ for (String name : fieldNames) {
+ if (name.equals(fieldName)) {
+ return index;
+ }
+ index++;
+ }
+ return -1;
+ }
+
+ public ARecordType getRecordType() {
+ return recordType;
+ }
+
+ public List<IJObject> getFields() {
+ return fields;
+ }
+
+ @Override
+ public IAObject getIAObject() {
+ if (value == null || numFieldsAdded > 0) {
+ value = new AMutableRecord(recordType, getIAObjectArray(fields));
+ }
+ return value;
+ }
+
+ public void close() {
+ if (numFieldsAdded > 0) {
+ int totalFields = fieldNames.size();
+ for (int i = 0; i < numFieldsAdded; i++) {
+ fieldNames.remove(totalFields - 1 - i);
+ fieldTypes.remove(totalFields - 1 - i);
+ fields.remove(totalFields - 1 - i);
+ }
+ numFieldsAdded = 0;
+ }
+ }
+
+ public List<Boolean> getOpenField() {
+ return openField;
+ }
+
+ public List<String> getFieldNames() {
+ return fieldNames;
+ }
+
+ public List<IAType> getFieldTypes() {
+ return fieldTypes;
+ }
+
+ }
+
+ public static class ByteArrayAccessibleInputStream extends ByteArrayInputStream {
+
+ public ByteArrayAccessibleInputStream(byte[] buf, int offset, int length) {
+ super(buf, offset, length);
+ }
+
+ public void setContent(byte[] buf, int offset, int length) {
+ this.buf = buf;
+ this.pos = offset;
+ this.count = Math.min(offset + length, buf.length);
+ this.mark = offset;
+ }
+
+ public byte[] getArray() {
+ return buf;
+ }
+
+ public int getPosition() {
+ return pos;
+ }
+
+ public int getCount() {
+ return count;
+ }
+
+ }
+
+ public static class ByteArrayAccessibleDataInputStream extends DataInputStream {
+
+ public ByteArrayAccessibleDataInputStream(ByteArrayAccessibleInputStream in) {
+ super(in);
+ }
+
+ public ByteArrayAccessibleInputStream getInputStream() {
+ return (ByteArrayAccessibleInputStream) in;
+ }
+
+ }
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JTypeTag.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JTypeTag.java
new file mode 100644
index 0000000..3b686f6
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JTypeTag.java
@@ -0,0 +1,12 @@
+package edu.uci.ics.asterix.external.library.java;
+
+public enum JTypeTag {
+
+ INT,
+ STRING,
+ LONG,
+ DOUBLE,
+ FLOAT,
+ LIST,
+ OBJECT
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/feed/managed/adapter/IManagedFeedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/feed/managed/adapter/IManagedFeedAdapter.java
deleted file mode 100644
index 87e8478..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/feed/managed/adapter/IManagedFeedAdapter.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.feed.managed.adapter;
-
-import java.util.Map;
-
-/**
- * Interface implemented by an adapter that can be controlled or managed by external
- * commands (stop,alter)
- */
-public interface IManagedFeedAdapter {
-
- /**
- * Discontinue the ingestion of data and end the feed.
- *
- * @throws Exception
- */
- public void stop();
-
- /**
- * Modify the adapter configuration parameters. This method is called
- * when the configuration parameters need to be modified while the adapter
- * is ingesting data in an active feed.
- *
- * @param properties
- * A HashMap containing the set of configuration parameters
- * that need to be altered.
- */
- public void alter(Map<String, String> properties);
-
-}
diff --git a/asterix-external-data/src/main/resources/feed_policy/basic.pol b/asterix-external-data/src/main/resources/feed_policy/basic.pol
new file mode 100644
index 0000000..8aed9f8
--- /dev/null
+++ b/asterix-external-data/src/main/resources/feed_policy/basic.pol
@@ -0,0 +1,8 @@
+feed.policy.name=Basic
+
+software.failure.persist.exception=false
+software.failure.continue.on.failure=false
+hardware.failure.auto.restart=false
+
+statistics.collect=false
+elastic=false
diff --git a/asterix-external-data/src/main/resources/feed_policy/basic_monitored.pol b/asterix-external-data/src/main/resources/feed_policy/basic_monitored.pol
new file mode 100644
index 0000000..efca65e
--- /dev/null
+++ b/asterix-external-data/src/main/resources/feed_policy/basic_monitored.pol
@@ -0,0 +1,11 @@
+feed.policy.name=Basic_Monitored
+
+software.failure.persist.exception=true
+software.failure.continue.on.failure=false
+hardware.failure.auto.restart=false
+
+statistics.collect=true
+statistics.collect.period=60
+statistics.collect.period.unit=sec
+
+elastic=false
diff --git a/asterix-external-data/src/main/resources/feed_policy/mission_critical.pol b/asterix-external-data/src/main/resources/feed_policy/mission_critical.pol
new file mode 100644
index 0000000..0a1da08
--- /dev/null
+++ b/asterix-external-data/src/main/resources/feed_policy/mission_critical.pol
@@ -0,0 +1,17 @@
+feed.policy.name=Mission_Critical
+
+software.failure.persist.exception=true
+software.failure.continue.on.failure=true
+hardware.failure.auto.restart=true
+
+statistics.collect=true
+statistics.collect.period=60
+statistics.collect.period.unit=sec
+
+elastic=true
+criterion.ingestion.rate.percent=true
+criterion.ingestion.rate.percent.delta=50
+
+criterion.ingestion.rate.absolute=true
+criterion.ingestion.rate.absolute.delta=2000
+criterion.ingestion.rate.absolute.unit=sec
diff --git a/asterix-external-data/src/main/resources/feed_policy/twitter.pol b/asterix-external-data/src/main/resources/feed_policy/twitter.pol
new file mode 100644
index 0000000..5fa507f
--- /dev/null
+++ b/asterix-external-data/src/main/resources/feed_policy/twitter.pol
@@ -0,0 +1,17 @@
+feed.policy.name=Twitter
+
+software.failure.persist.exception=true
+software.failure.continue.on.failure=true
+hardware.failure.auto.restart=false
+
+statistics.collect=true
+statistics.collect.period=60
+statistics.collect.period.unit=sec
+
+elastic=true
+criterion.ingestion.rate.percent=true
+criterion.ingestion.rate.percent.delta=50
+
+criterion.ingestion.rate.absolute=true
+criterion.ingestion.rate.absolute.delta=2000
+criterion.ingestion.rate.absolute.unit=sec
diff --git a/asterix-external-data/src/main/resources/schema/library.xsd b/asterix-external-data/src/main/resources/schema/library.xsd
new file mode 100644
index 0000000..3dc4659
--- /dev/null
+++ b/asterix-external-data/src/main/resources/schema/library.xsd
@@ -0,0 +1,42 @@
+<?xml version="1.0" encoding="ISO-8859-1" ?>
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:lib="library" targetNamespace="library" elementFormDefault="qualified">
+
+<!-- definition of simple types -->
+<xs:element name="language" type="xs:string"/>
+<xs:element name="name" type="xs:string"/>
+<xs:element name="arguments" type="xs:string"/>
+<xs:element name="return_type" type="xs:string"/>
+<xs:element name="function_type" type="xs:string"/>
+<xs:element name="definition" type="xs:string"/>
+
+<!-- definition of complex elements -->
+<xs:element name="function">
+ <xs:complexType>
+ <xs:sequence>
+ <xs:element ref="lib:name"/>
+ <xs:element ref="lib:function_type"/>
+ <xs:element ref="lib:arguments"/>
+ <xs:element ref="lib:return_type"/>
+ <xs:element ref="lib:definition"/>
+ </xs:sequence>
+ </xs:complexType>
+</xs:element>
+
+<xs:element name="functions">
+ <xs:complexType>
+ <xs:sequence>
+ <xs:element ref="lib:function" maxOccurs="unbounded"/>
+ </xs:sequence>
+ </xs:complexType>
+</xs:element>
+
+<xs:element name="library">
+ <xs:complexType>
+ <xs:sequence>
+ <xs:element ref="lib:language"/>
+ <xs:element ref="lib:functions" minOccurs="0"/>
+ </xs:sequence>
+ </xs:complexType>
+</xs:element>
+
+</xs:schema>
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AllTypesFactory.java
similarity index 64%
copy from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
copy to asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AllTypesFactory.java
index 98d72f4..f095321 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AllTypesFactory.java
@@ -3,28 +3,25 @@
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* you may obtain a copy of the License from
- *
+ *
* http://www.apache.org/licenses/LICENSE-2.0
- *
+ *
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.installer.service;
+package edu.uci.ics.asterix.external.library;
-public class ServiceProvider {
+import edu.uci.ics.asterix.external.library.IExternalFunction;
+import edu.uci.ics.asterix.external.library.IFunctionFactory;
- public static ServiceProvider INSTANCE = new ServiceProvider();
- private static ILookupService lookupService = new ZooKeeperService();
-
- private ServiceProvider() {
+public class AllTypesFactory implements IFunctionFactory {
+ @Override
+ public IExternalFunction getExternalFunction() {
+ return new AllTypesFunction();
}
- public ILookupService getLookupService() {
- return lookupService;
- }
-
}
diff --git a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AllTypesFunction.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AllTypesFunction.java
new file mode 100644
index 0000000..c5063db
--- /dev/null
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AllTypesFunction.java
@@ -0,0 +1,115 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.external.library;
+
+import edu.uci.ics.asterix.external.library.IExternalScalarFunction;
+import edu.uci.ics.asterix.external.library.IFunctionHelper;
+import edu.uci.ics.asterix.external.library.java.JObjects.JBoolean;
+import edu.uci.ics.asterix.external.library.java.JObjects.JCircle;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDate;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDateTime;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDouble;
+import edu.uci.ics.asterix.external.library.java.JObjects.JDuration;
+import edu.uci.ics.asterix.external.library.java.JObjects.JFloat;
+import edu.uci.ics.asterix.external.library.java.JObjects.JInt;
+import edu.uci.ics.asterix.external.library.java.JObjects.JLine;
+import edu.uci.ics.asterix.external.library.java.JObjects.JOrderedList;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPoint;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPoint3D;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPolygon;
+import edu.uci.ics.asterix.external.library.java.JObjects.JRecord;
+import edu.uci.ics.asterix.external.library.java.JObjects.JString;
+import edu.uci.ics.asterix.external.library.java.JObjects.JTime;
+import edu.uci.ics.asterix.external.library.java.JObjects.JUnorderedList;
+import edu.uci.ics.asterix.external.library.java.JTypeTag;
+
+public class AllTypesFunction implements IExternalScalarFunction {
+
+ private JOrderedList newFieldList;
+
+ @Override
+ public void initialize(IFunctionHelper functionHelper) {
+ newFieldList = new JOrderedList(functionHelper.getObject(JTypeTag.INT));
+ }
+
+ @Override
+ public void deinitialize() {
+ }
+
+ @Override
+ public void evaluate(IFunctionHelper functionHelper) throws Exception {
+ newFieldList.clear();
+ JRecord inputRecord = (JRecord) functionHelper.getArgument(0);
+ JInt id = (JInt) inputRecord.getValueByName("id");
+ JString name = (JString) inputRecord.getValueByName("name");
+ JFloat age = (JFloat) inputRecord.getValueByName("age");
+ JDouble salary = (JDouble) inputRecord.getValueByName("salary");
+ JBoolean married = (JBoolean) inputRecord.getValueByName("married");
+ JUnorderedList interest = (JUnorderedList) inputRecord
+ .getValueByName("interests");
+ JOrderedList children = (JOrderedList) inputRecord
+ .getValueByName("children");
+ JRecord address = (JRecord) inputRecord.getValueByName("address");
+ JDate dob = (JDate) inputRecord.getValueByName("dob");
+ JTime time = (JTime) inputRecord.getValueByName("time");
+ JDateTime dateTime = (JDateTime) inputRecord.getValueByName("datetime");
+ JDuration duration = (JDuration) inputRecord.getValueByName("duration");
+ JPoint location2d = (JPoint) inputRecord.getValueByName("location2d");
+ JPoint3D location3d = (JPoint3D) inputRecord
+ .getValueByName("location3d");
+ JLine line = (JLine) inputRecord.getValueByName("line");
+ JPolygon polygon = (JPolygon) inputRecord.getValueByName("polygon");
+ JCircle circle = (JCircle) inputRecord.getValueByName("circle");
+
+ JRecord result = (JRecord) functionHelper.getResultObject();
+ result.setField("id", id);
+ result.setField("name", name);
+ result.setField("age", age);
+ result.setField("salary", salary);
+ result.setField("married", married);
+ result.setField("interests", interest);
+ result.setField("children", children);
+ JInt zipCode = (JInt) functionHelper.getObject(JTypeTag.INT);
+ zipCode.setValue(92841);
+ address.addField("Zipcode", zipCode);
+ result.setField("address", address);
+ result.setField("dob", dob);
+ result.setField("time", time);
+ result.setField("datetime", dateTime);
+ result.setField("duration", duration);
+ result.setField("location2d", location2d);
+ result.setField("location3d", location3d);
+ result.setField("line", line);
+ result.setField("polygon", polygon);
+ result.setField("circle", circle);
+
+ JString newFieldString = (JString) functionHelper
+ .getObject(JTypeTag.STRING);
+ newFieldString.setValue("processed");
+ result.addField("status", newFieldString);
+
+ /*
+ * JString element = (JString)
+ * functionHelper.getObject(JTypeTag.STRING); element.setValue("raman");
+ * newFieldList.add(element); result.addField("mylist", newFieldList);
+ */
+
+ JString newFieldString2 = (JString) functionHelper
+ .getObject(JTypeTag.STRING);
+ newFieldString2.setValue("this is working");
+ result.addField("working", newFieldString);
+ functionHelper.setResult(result);
+ }
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/CapitalFinderFactory.java
similarity index 63%
copy from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
copy to asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/CapitalFinderFactory.java
index 98d72f4..872b542 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/CapitalFinderFactory.java
@@ -3,28 +3,25 @@
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* you may obtain a copy of the License from
- *
+ *
* http://www.apache.org/licenses/LICENSE-2.0
- *
+ *
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.installer.service;
+package edu.uci.ics.asterix.external.library;
-public class ServiceProvider {
+import edu.uci.ics.asterix.external.library.IExternalScalarFunction;
+import edu.uci.ics.asterix.external.library.IFunctionFactory;
- public static ServiceProvider INSTANCE = new ServiceProvider();
- private static ILookupService lookupService = new ZooKeeperService();
-
- private ServiceProvider() {
+public class CapitalFinderFactory implements IFunctionFactory {
+ @Override
+ public IExternalScalarFunction getExternalFunction() {
+ return new CapitalFinderFunction();
}
- public ILookupService getLookupService() {
- return lookupService;
- }
-
}
diff --git a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/CapitalFinderFunction.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/CapitalFinderFunction.java
new file mode 100644
index 0000000..7953b62
--- /dev/null
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/CapitalFinderFunction.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.external.library;
+
+import java.io.InputStream;
+import java.util.Properties;
+
+import edu.uci.ics.asterix.external.library.java.JObjects.JRecord;
+import edu.uci.ics.asterix.external.library.java.JObjects.JString;
+import edu.uci.ics.asterix.external.library.java.JTypeTag;
+
+public class CapitalFinderFunction implements IExternalScalarFunction {
+
+ private static Properties capitalList;
+ private static final String NOT_FOUND = "NOT_FOUND";
+ private JString capital;
+
+ @Override
+ public void deinitialize() {
+ System.out.println(" De Initialized");
+ }
+
+ @Override
+ public void evaluate(IFunctionHelper functionHelper) throws Exception {
+ JString country = ((JString) functionHelper.getArgument(0));
+ JRecord record = (JRecord) functionHelper.getResultObject();
+ // ((JString) record.getValueByName("country")).setValue(country.getValue());
+ String capitalCity = capitalList.getProperty(country.getValue(), NOT_FOUND);
+ // ((JString) record.getValueByName("capital")).setValue(capitalCity);
+ capital.setValue(capitalCity);
+
+ record.setField("country", country);
+ record.setField("capital", capital);
+ functionHelper.setResult(record);
+ }
+
+ @Override
+ public void initialize(IFunctionHelper functionHelper) throws Exception {
+ InputStream in = CapitalFinderFunction.class.getClassLoader().getResourceAsStream("data/countriesCapitals.txt");
+ capitalList = new Properties();
+ capitalList.load(in);
+ capital = (JString) functionHelper.getObject(JTypeTag.STRING);
+ }
+
+}
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/xml/PatternParser.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/EchoDelayFactory.java
similarity index 72%
copy from asterix-events/src/main/java/edu/uci/ics/asterix/event/xml/PatternParser.java
copy to asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/EchoDelayFactory.java
index af46e63..d15d661 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/xml/PatternParser.java
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/EchoDelayFactory.java
@@ -3,21 +3,23 @@
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* you may obtain a copy of the License from
- *
+ *
* http://www.apache.org/licenses/LICENSE-2.0
- *
+ *
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.event.xml;
+package edu.uci.ics.asterix.external.library;
-public class PatternParser {
- public static void parsePattern(String path){
-
- }
+public class EchoDelayFactory implements IFunctionFactory {
+
+ @Override
+ public IExternalScalarFunction getExternalFunction() {
+ return new EchoDelayFunction();
+ }
+
}
-
diff --git a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/EchoDelayFunction.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/EchoDelayFunction.java
new file mode 100644
index 0000000..5f9be77
--- /dev/null
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/EchoDelayFunction.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.external.library;
+
+import java.util.Random;
+
+import edu.uci.ics.asterix.external.library.java.JObjects.JRecord;
+
+public class EchoDelayFunction implements IExternalScalarFunction {
+
+ private Random rand = new Random();
+ private long sleepIntervalMin;
+ private long sleepIntervalMax;
+ private int range;
+
+ @Override
+ public void initialize(IFunctionHelper functionHelper) {
+ sleepIntervalMin = 50;
+ sleepIntervalMax = 100;
+ range = (new Long(sleepIntervalMax - sleepIntervalMin)).intValue();
+ }
+
+ @Override
+ public void deinitialize() {
+ }
+
+ @Override
+ public void evaluate(IFunctionHelper functionHelper) throws Exception {
+ JRecord inputRecord = (JRecord) functionHelper.getArgument(0);
+ long sleepInterval = rand.nextInt(range);
+ // Thread.sleep(5);
+ functionHelper.setResult(inputRecord);
+ }
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/ParseTweetFactory.java
similarity index 63%
copy from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
copy to asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/ParseTweetFactory.java
index 98d72f4..d868f20 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/ParseTweetFactory.java
@@ -3,28 +3,25 @@
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* you may obtain a copy of the License from
- *
+ *
* http://www.apache.org/licenses/LICENSE-2.0
- *
+ *
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.installer.service;
+package edu.uci.ics.asterix.external.library;
-public class ServiceProvider {
+import edu.uci.ics.asterix.external.library.IExternalScalarFunction;
+import edu.uci.ics.asterix.external.library.IFunctionFactory;
- public static ServiceProvider INSTANCE = new ServiceProvider();
- private static ILookupService lookupService = new ZooKeeperService();
-
- private ServiceProvider() {
+public class ParseTweetFactory implements IFunctionFactory {
+ @Override
+ public IExternalScalarFunction getExternalFunction() {
+ return new ParseTweetFunction();
}
- public ILookupService getLookupService() {
- return lookupService;
- }
-
}
diff --git a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/ParseTweetFunction.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/ParseTweetFunction.java
new file mode 100644
index 0000000..de13e59
--- /dev/null
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/ParseTweetFunction.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.external.library;
+
+import edu.uci.ics.asterix.external.library.java.JObjects.JOrderedList;
+import edu.uci.ics.asterix.external.library.java.JObjects.JRecord;
+import edu.uci.ics.asterix.external.library.java.JObjects.JString;
+import edu.uci.ics.asterix.external.library.java.JTypeTag;
+
+public class ParseTweetFunction implements IExternalScalarFunction {
+
+ private JOrderedList list = null;
+
+ @Override
+ public void initialize(IFunctionHelper functionHelper) {
+ list = new JOrderedList(functionHelper.getObject(JTypeTag.STRING));
+ }
+
+ @Override
+ public void deinitialize() {
+ }
+
+ @Override
+ public void evaluate(IFunctionHelper functionHelper) throws Exception {
+ list.clear();
+ JRecord inputRecord = (JRecord) functionHelper.getArgument(0);
+ JString id = (JString) inputRecord.getValueByName("id");
+ JString text = (JString) inputRecord.getValueByName("text");
+
+ String[] tokens = text.getValue().split(" ");
+ for (String tk : tokens) {
+ if (tk.startsWith("#")) {
+ JString newField = (JString) functionHelper.getObject(JTypeTag.STRING);
+ newField.setValue(tk);
+ list.add(newField);
+ }
+ }
+ JRecord result = (JRecord) functionHelper.getResultObject();
+ result.setField("id", id);
+ result.setField("username", inputRecord.getValueByName("username"));
+ result.setField("location", inputRecord.getValueByName("location"));
+ result.setField("text", text);
+ result.setField("timestamp", inputRecord.getValueByName("timestamp"));
+ result.setField("topics", list);
+ functionHelper.setResult(result);
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedMessage.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/SumFactory.java
similarity index 64%
copy from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedMessage.java
copy to asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/SumFactory.java
index dcef2c8..eabc6b5 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedMessage.java
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/SumFactory.java
@@ -3,26 +3,25 @@
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* you may obtain a copy of the License from
- *
+ *
* http://www.apache.org/licenses/LICENSE-2.0
- *
+ *
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.external.feed.lifecycle;
+package edu.uci.ics.asterix.external.library;
+
+import edu.uci.ics.asterix.external.library.IExternalScalarFunction;
+import edu.uci.ics.asterix.external.library.IFunctionFactory;
-import java.io.Serializable;
+public class SumFactory implements IFunctionFactory {
-public interface IFeedMessage extends Serializable {
-
- public enum MessageType {
- STOP,
- ALTER,
- }
-
- public MessageType getMessageType();
+ @Override
+ public IExternalScalarFunction getExternalFunction() {
+ return new SumFunction();
+ }
}
diff --git a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/SumFunction.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/SumFunction.java
new file mode 100644
index 0000000..0a4eca6
--- /dev/null
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/SumFunction.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.external.library;
+
+import edu.uci.ics.asterix.external.library.IExternalScalarFunction;
+import edu.uci.ics.asterix.external.library.IFunctionHelper;
+import edu.uci.ics.asterix.external.library.java.JObjects.JInt;
+
+public class SumFunction implements IExternalScalarFunction {
+
+ private JInt result;
+
+ @Override
+ public void deinitialize() {
+ System.out.println(" De Initialized");
+ }
+
+ @Override
+ public void evaluate(IFunctionHelper functionHelper) throws Exception {
+ int arg0 = ((JInt) functionHelper.getArgument(0)).getValue();
+ int arg1 = ((JInt) functionHelper.getArgument(1)).getValue();
+ result.setValue(arg0 + arg1);
+ functionHelper.setResult(result);
+ }
+
+ @Override
+ public void initialize(IFunctionHelper functionHelper) {
+ result = (JInt) functionHelper.getResultObject();
+ }
+
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/UpperCaseFactory.java
similarity index 64%
copy from asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
copy to asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/UpperCaseFactory.java
index 98d72f4..2c5f607 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ServiceProvider.java
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/UpperCaseFactory.java
@@ -3,28 +3,25 @@
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* you may obtain a copy of the License from
- *
+ *
* http://www.apache.org/licenses/LICENSE-2.0
- *
+ *
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.installer.service;
+package edu.uci.ics.asterix.external.library;
-public class ServiceProvider {
+import edu.uci.ics.asterix.external.library.IExternalFunction;
+import edu.uci.ics.asterix.external.library.IFunctionFactory;
- public static ServiceProvider INSTANCE = new ServiceProvider();
- private static ILookupService lookupService = new ZooKeeperService();
-
- private ServiceProvider() {
+public class UpperCaseFactory implements IFunctionFactory {
+ @Override
+ public IExternalFunction getExternalFunction() {
+ return new UpperCaseFunction();
}
- public ILookupService getLookupService() {
- return lookupService;
- }
-
}
diff --git a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/UpperCaseFunction.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/UpperCaseFunction.java
new file mode 100644
index 0000000..e3d1c9c
--- /dev/null
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/UpperCaseFunction.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.external.library;
+
+import java.util.Random;
+
+import edu.uci.ics.asterix.external.library.IExternalScalarFunction;
+import edu.uci.ics.asterix.external.library.IFunctionHelper;
+import edu.uci.ics.asterix.external.library.java.JTypeTag;
+import edu.uci.ics.asterix.external.library.java.JObjects.JInt;
+import edu.uci.ics.asterix.external.library.java.JObjects.JRecord;
+import edu.uci.ics.asterix.external.library.java.JObjects.JString;
+
+/**
+ * Accepts an input record of type Open{ id: int32, text: string }
+ *
+ * Converts the text field into upper case and appends an additional field -
+ * "substring" with value as a random substring of the text field.
+ *
+ * Return Type Open{ id: int32, text: string }
+ *
+ */
+public class UpperCaseFunction implements IExternalScalarFunction {
+
+ private Random random;
+
+ @Override
+ public void initialize(IFunctionHelper functionHelper) {
+ random = new Random();
+ }
+
+ @Override
+ public void deinitialize() {
+ }
+
+ @Override
+ public void evaluate(IFunctionHelper functionHelper) throws Exception {
+ JRecord inputRecord = (JRecord) functionHelper.getArgument(0);
+ JInt id = (JInt) inputRecord.getValueByName("id");
+ id.setValue(id.getValue() * -1); // for maintaining uniqueness
+ // constraint in the case when
+ // output is re-inserted into source
+ // dataset
+ JString text = (JString) inputRecord.getValueByName("text");
+ text.setValue(text.getValue().toUpperCase());
+ JRecord result = (JRecord) functionHelper.getResultObject();
+ result.setField("id", id);
+ result.setField("text", text);
+ JString newField = (JString) functionHelper.getObject(JTypeTag.STRING);
+ newField.setValue(text.getValue().substring(
+ random.nextInt(text.getValue().length())));
+ result.addField("substring", newField);
+ functionHelper.setResult(result);
+ }
+}
diff --git a/asterix-external-data/src/test/resources/data/countriesCapitals.txt b/asterix-external-data/src/test/resources/data/countriesCapitals.txt
new file mode 100644
index 0000000..def17ed
--- /dev/null
+++ b/asterix-external-data/src/test/resources/data/countriesCapitals.txt
@@ -0,0 +1,189 @@
+United\ Arab\ Emirates=Abu Dhabi
+Nigeria=Abuja
+Ghana=Accra
+Ethiopia=Addis Ababa
+Algeria=Algiers
+Niue=Alofi
+Jordan=Amman
+Netherlands=Amsterdam
+Andorra=Andorra la Vella
+Turkey=Ankara
+Madagascar=Antananarivo
+Samoa=Apia
+Turkmenistan=Ashgabat
+Eritrea=Asmara
+Kazakhstan=Astana
+Paraguay=Asunción
+Greece=Athens
+Iraq=Baghdad
+Azerbaijan=Baku
+Mali=Bamako
+Brunei=Bandar Seri Begawan
+Thailand=Bangkok
+Gambia=Banjul
+China=Beijing
+Lebanon=Beirut
+Serbia=Belgrade
+Belize=Belmopan
+Germany=Berlin
+Switzerland=Bern
+Kyrgyzstan=Bishkek
+Guinea-Bissau=Bissau
+Colombia=Bogotá
+Brazil=Brasília
+Slovakia=Bratislava
+Barbados=Bridgetown
+Belgium=Brussels
+Romania=Bucharest
+Hungary=Budapest
+Argentina=Buenos Aires
+Burundi=Bujumbura
+Egypt=Cairo
+Australia=Canberra
+Venezuela=Caracas
+Wales=Cardiff
+Moldova=Chisinau
+Guinea=Conakry
+Denmark=Copenhagen
+Senegal=Dakar
+Syria=Damascus
+Bangladesh=Dhaka
+Djibouti=Djibouti
+Qatar=Doha
+Ireland=Dublin
+Tajikistan=Dushanbe
+Scotland=Edinburgh
+Botswana=Gaborone
+Guyana=Georgetown
+Gibraltar=Gibraltar
+Guatemala=Guatemala City
+Guam=Hagåtña
+Bermuda=Hamilton
+Vietnam=Hanoi
+Zimbabwe=Harare
+Cuba=Havana
+Finland=Helsinki
+Pakistan=Islamabad
+Indonesia=Jakarta
+Afghanistan=Kabul
+Uganda=Kampala
+Nepal=Kathmandu
+Sudan=Khartoum
+Ukraine=Kiev
+Rwanda=Kigali
+Jamaica=Kingston
+Malaysia=Kuala Lumpur
+Kuwait=Kuwait City
+Gabon=Libreville
+Malawi=Lilongwe
+Peru=Lima
+Portugal=Lisbon
+Slovenia=Ljubljana
+Togo=Lome
+England=London
+Angola=Luanda
+Zambia=Lusaka
+Luxembourg=Luxembourg
+Spain=Madrid
+Marshall Islands=Majuro
+Equatorial Guinea=Malabo
+Maldives=Malé
+Nicaragua=Managua
+Bahrain=Manama
+Philippines=Manila
+Mozambique=Maputo
+Saint Martin=Marigot
+Lesotho=Maseru
+Wallis and Futuna=Mata-Utu
+Palau=Melekeok
+Mexico=Mexico City
+Belarus=Minsk
+Somalia=Mogadishu
+Monaco=Monaco
+Liberia=Monrovia
+Uruguay=Montevideo
+Comoros=Moroni
+Russia=Moscow
+Oman=Muscat
+Kenya=Nairobi
+Bahamas=Nassau
+Myanmar=Naypyidaw
+Chad=NDjamena
+India=New Delhi
+Niger=Niamey
+Cyprus=Nicosia
+Mauritania=Nouakchott
+Greenland=Nuuk
+Aruba=Oranjestad
+Norway=Oslo
+Canada=Ottawa
+Panama=Panama City
+Suriname=Paramaribo
+France=Paris
+Cambodia=Phnom Penh
+Montserrat=Brades Estate (de facto)
+Montenegro=Podgorica
+Mauritius=Port Louis
+Vanuatu=Port Vila
+Haiti=Port-au-Prince
+Benin=Cotonou (de facto)
+Czech Republic=Prague
+Cape Verde=Praia
+South\ Africa=Cape Town
+North\ Korea=Pyongyang
+Ecuador=Quito
+Morocco=Rabat
+Iceland=Reykjavík
+Latvia=Riga
+Saudi\ Arabia=Riyadh
+Italy=Rome
+Dominica=Roseau
+Costa\ Rica=San José
+Puerto\ Rico=San Juan
+San\ Marino=San Marino
+El\ Salvador=San Salvador
+Yemen=Sanaa
+Chile=Santiago
+Dominican\ Republic=Santo Domingo
+Bosnia\ and\ Herzegovina=Sarajevo
+South\ Korea=Seoul
+Singapore=Singapore
+Macedonia=Skopje
+Bulgaria=Sofia
+Sri\ Lanka=Colombo
+Grenada=St. George's
+Jersey=St. Helier
+Guernsey=St. Peter Port
+Sweden=Stockholm
+Bolivia=La Paz
+Abkhazia=Sukhumi
+Fiji=Suva
+Taiwan=Taipei
+Estonia=Tallinn
+Kiribati=Tarawa
+Uzbekistan=Tashkent
+Georgia=Tbilisi
+Honduras=Tegucigalpa
+Iran=Tehran
+Bhutan=Thimphu
+Albania=Tirana
+Transnistria=Tiraspol
+Japan=Tokyo
+Libya=Tripoli
+Tunisia=Tunis
+Mongolia=Ulan Bator
+Liechtenstein=Vaduz
+Malta=Valletta
+Anguilla=The Valley
+Vatican\ City=Vatican City
+Seychelles=Victoria
+Austria=Vienna
+Laos=Vientiane
+Lithuania=Vilnius
+Poland=Warsaw
+United\ States=Washington D.C.
+New\ Zealand=Wellington
+Namibia=Windhoek
+Nauru=Yaren (de facto)
+Armenia=Yerevan
+Croatia=Zagreb
diff --git a/asterix-external-data/src/test/resources/text_functions.xml b/asterix-external-data/src/test/resources/text_functions.xml
new file mode 100644
index 0000000..6d00029
--- /dev/null
+++ b/asterix-external-data/src/test/resources/text_functions.xml
@@ -0,0 +1,53 @@
+<library xmlns="library">
+ <language>JAVA</language>
+ <functions>
+ <function>
+ <function_type>SCALAR</function_type>
+ <name>parseTweet</name>
+ <arguments>TweetType</arguments>
+ <return_type>TweetType</return_type>
+ <definition>edu.uci.ics.asterix.external.library.ParseTweetFactory
+ </definition>
+ </function>
+ <function>
+ <function_type>SCALAR</function_type>
+ <name>mysum</name>
+ <arguments>AINT32,AINT32</arguments>
+ <return_type>AINT32</return_type>
+ <definition>edu.uci.ics.asterix.external.library.SumFactory
+ </definition>
+ </function>
+ <function>
+ <function_type>SCALAR</function_type>
+ <name>getCapital</name>
+ <arguments>ASTRING</arguments>
+ <return_type>CountryCapitalType</return_type>
+ <definition>edu.uci.ics.asterix.external.library.CapitalFinderFactory
+ </definition>
+ </function>
+ <function>
+ <function_type>SCALAR</function_type>
+ <name>toUpper</name>
+ <arguments>TextType</arguments>
+ <return_type>TextType</return_type>
+ <definition>edu.uci.ics.asterix.external.library.UpperCaseFactory
+ </definition>
+ </function>
+ <function>
+ <function_type>SCALAR</function_type>
+ <name>allTypes</name>
+ <arguments>AllType</arguments>
+ <return_type>AllType</return_type>
+ <definition>edu.uci.ics.asterix.external.library.AllTypesFactory
+ </definition>
+ </function>
+ <function>
+ <function_type>SCALAR</function_type>
+ <name>echoDelay</name>
+ <arguments>TweetMessageType</arguments>
+ <return_type>TweetMessageType</return_type>
+ <definition>edu.uci.ics.asterix.external.library.EchoDelayFactory
+ </definition>
+ </function>
+ </functions>
+</library>
diff --git a/asterix-installer/pom.xml b/asterix-installer/pom.xml
index 242f873..ad515ec 100644
--- a/asterix-installer/pom.xml
+++ b/asterix-installer/pom.xml
@@ -1,25 +1,21 @@
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
- <parent>
- <artifactId>asterix</artifactId>
- <groupId>edu.uci.ics.asterix</groupId>
- <version>0.8.1-SNAPSHOT</version>
- </parent>
- <artifactId>asterix-installer</artifactId>
+<!-- ! Copyright 2009-2013 by The Regents of the University of California
+ ! Licensed under the Apache License, Version 2.0 (the "License"); ! you may
+ not use this file except in compliance with the License. ! you may obtain
+ a copy of the License from ! ! http://www.apache.org/licenses/LICENSE-2.0
+ ! ! Unless required by applicable law or agreed to in writing, software !
+ distributed under the License is distributed on an "AS IS" BASIS, ! WITHOUT
+ WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. ! See the
+ License for the specific language governing permissions and ! limitations
+ under the License. ! -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <artifactId>asterix</artifactId>
+ <groupId>edu.uci.ics.asterix</groupId>
+ <version>0.8.1-SNAPSHOT</version>
+ </parent>
+ <artifactId>asterix-installer</artifactId>
<properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
</properties>
@@ -31,11 +27,10 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
</configuration>
</plugin>
-
<plugin>
<groupId>org.jvnet.jaxb2.maven2</groupId>
<artifactId>maven-jaxb2-plugin</artifactId>
@@ -76,10 +71,6 @@
<goal>generate</goal>
</goals>
<configuration>
- <args>
- <arg>-Xsetters</arg>
- <arg>-Xvalue-constructor</arg>
- </args>
<schemaDirectory>src/main/resources/schema</schemaDirectory>
<schemaIncludes>
<include>cluster.xsd</include>
@@ -174,16 +165,9 @@
<groupId>edu.uci.ics.asterix</groupId>
<artifactId>asterix-common</artifactId>
<version>0.8.1-SNAPSHOT</version>
- <type>jar</type>
- <scope>compile</scope>
+ <type>test-jar</type>
+ <scope>test</scope>
</dependency>
- <dependency>
- <groupId>edu.uci.ics.asterix</groupId>
- <artifactId>asterix-common</artifactId>
- <version>0.8.1-SNAPSHOT</version>
- <type>test-jar</type>
- <scope>test</scope>
- </dependency>
<dependency>
<groupId>edu.uci.ics.asterix</groupId>
<artifactId>asterix-server</artifactId>
@@ -191,12 +175,11 @@
<type>zip</type>
<classifier>binary-assembly</classifier>
</dependency>
- <dependency>
- <groupId>edu.uci.ics.asterix</groupId>
- <artifactId>asterix-test-framework</artifactId>
- <version>0.8.1-SNAPSHOT</version>
- <scope>test</scope>
- </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.asterix</groupId>
+ <artifactId>asterix-test-framework</artifactId>
+ <version>0.8.1-SNAPSHOT</version>
+ <scope>test</scope>
+ </dependency>
</dependencies>
-
</project>
diff --git a/asterix-installer/src/main/assembly/binary-assembly.xml b/asterix-installer/src/main/assembly/binary-assembly.xml
index d42c5ae..930f686 100644
--- a/asterix-installer/src/main/assembly/binary-assembly.xml
+++ b/asterix-installer/src/main/assembly/binary-assembly.xml
@@ -13,112 +13,108 @@
! limitations under the License.
!-->
<assembly>
- <id>binary-assembly</id>
- <formats>
- <format>zip</format>
- <format>dir</format>
- </formats>
- <includeBaseDirectory>false</includeBaseDirectory>
- <fileSets>
- <fileSet>
- <directory>src/main/resources/conf</directory>
- <outputDirectory>conf</outputDirectory>
- </fileSet>
- <fileSet>
- <directory>src/main/resources/clusters</directory>
- <outputDirectory>clusters</outputDirectory>
- </fileSet>
- <fileSet>
- <directory>src/main/resources/zookeeper</directory>
- <fileMode>0755</fileMode>
- <outputDirectory>.installer/zookeeper/bin</outputDirectory>
- </fileSet>
- <fileSet>
- <directory>src/main/resources/scripts</directory>
- <fileMode>0755</fileMode>
- <includes>
- <include>managix</include>
- </includes>
- <outputDirectory>bin</outputDirectory>
- </fileSet>
- <fileSet>
- <directory>src/main/resources/scripts</directory>
- <fileMode>0755</fileMode>
- <excludes>
- <exclude>managix</exclude>
- </excludes>
- <outputDirectory>.installer/scripts</outputDirectory>
- </fileSet>
- <fileSet>
- <directory>src/main/resources/hadoop-0.20.2</directory>
- <outputDirectory>.installer/hadoop-0.20.2</outputDirectory>
- <fileMode>0755</fileMode>
- </fileSet>
- <fileSet>
- <directory>target</directory>
- <outputDirectory>lib</outputDirectory>
- <includes>
- <include>*.jar</include>
- </includes>
- </fileSet>
- <fileSet>
- <directory>../asterix-events/src/main/resources/events</directory>
- <outputDirectory>.installer/eventrix/events</outputDirectory>
- <fileMode>0755</fileMode>
- </fileSet>
- <fileSet>
- <directory>../asterix-events/src/main/resources/scripts</directory>
- <outputDirectory>.installer/eventrix/scripts</outputDirectory>
- </fileSet>
- </fileSets>
- <dependencySets>
- <dependencySet>
- <includes>
- <include>log4j:log4j</include>
- <include>edu.uci.ics.asterix:asterix-events</include>
- <include>edu.uci.ics.asterix:asterix-common</include>
- <include>org.apache.zookeeper:zookeeper</include>
- <include>args4j:args4j</include>
- <include>log4j:log4j</include>
- <include>commons-io:commons-io</include>
- <include>org.slf4j:slf4j-api</include>
- <include>org.slf4j:slf4j-log4j12</include>
- </includes>
- <unpack>false</unpack>
- <outputDirectory>lib</outputDirectory>
- </dependencySet>
- <dependencySet>
- <includes>
- <include>org.apache.hadoop:hadoop-core</include>
- <include>commons-cli:commons-cli</include>
- <include>commons-logging:commons-logging</include>
- </includes>
- <unpack>false</unpack>
- <outputDirectory>.installer/hadoop-0.20.2/lib</outputDirectory>
- </dependencySet>
- <dependencySet>
- <includes>
- <include>org.apache.zookeeper:zookeeper</include>
- <include>log4j:log4j</include>
- <include>org.slf4j:slf4j-api</include>
- </includes>
- <unpack>false</unpack>
- <outputDirectory>.installer/zookeeper/lib</outputDirectory>
- </dependencySet>
- <dependencySet>
- <outputDirectory>asterix</outputDirectory>
- <includes>
- <include>asterix-server*</include>
- </includes>
- <useTransitiveDependencies>false</useTransitiveDependencies>
- </dependencySet>
- <dependencySet>
- <outputDirectory>.installer/eventrix</outputDirectory>
- <includes>
- <include>asterix-events*</include>
- </includes>
- <unpack>false</unpack>
- <useTransitiveDependencies>false</useTransitiveDependencies>
- </dependencySet>
- </dependencySets>
-</assembly>
+ <id>binary-assembly</id>
+ <formats>
+ <format>zip</format>
+ <format>dir</format>
+ </formats>
+ <includeBaseDirectory>false</includeBaseDirectory>
+ <fileSets>
+ <fileSet>
+ <directory>src/main/resources/conf</directory>
+ <outputDirectory>conf</outputDirectory>
+ </fileSet>
+ <fileSet>
+ <directory>src/main/resources/clusters</directory>
+ <outputDirectory>clusters</outputDirectory>
+ </fileSet>
+ <fileSet>
+ <directory>src/main/resources/zookeeper</directory>
+ <fileMode>0755</fileMode>
+ <outputDirectory>.installer/zookeeper/bin</outputDirectory>
+ </fileSet>
+ <fileSet>
+ <directory>src/main/resources/scripts</directory>
+ <fileMode>0755</fileMode>
+ <includes>
+ <include>managix</include>
+ </includes>
+ <outputDirectory>bin</outputDirectory>
+ </fileSet>
+ <fileSet>
+ <directory>src/main/resources/scripts</directory>
+ <fileMode>0755</fileMode>
+ <excludes>
+ <exclude>managix</exclude>
+ </excludes>
+ <outputDirectory>.installer/scripts</outputDirectory>
+ </fileSet>
+ <fileSet>
+ <directory>src/main/resources/hadoop-0.20.2</directory>
+ <outputDirectory>.installer/hadoop-0.20.2</outputDirectory>
+ <fileMode>0755</fileMode>
+ </fileSet>
+ <fileSet>
+ <directory>target</directory>
+ <outputDirectory>lib</outputDirectory>
+ <includes>
+ <include>*.jar</include>
+ </includes>
+ </fileSet>
+ <fileSet>
+ <directory>../asterix-events/src/main/resources/events</directory>
+ <outputDirectory>.installer/events</outputDirectory>
+ <fileMode>0755</fileMode>
+ </fileSet>
+ </fileSets>
+ <dependencySets>
+ <dependencySet>
+ <includes>
+ <include>log4j:log4j</include>
+ <include>edu.uci.ics.asterix:asterix-events</include>
+ <include>edu.uci.ics.asterix:asterix-common</include>
+ <include>org.apache.zookeeper:zookeeper</include>
+ <include>args4j:args4j</include>
+ <include>log4j:log4j</include>
+ <include>commons-io:commons-io</include>
+ <include>org.slf4j:slf4j-api</include>
+ <include>org.slf4j:slf4j-log4j12</include>
+ </includes>
+ <unpack>false</unpack>
+ <outputDirectory>lib</outputDirectory>
+ </dependencySet>
+ <dependencySet>
+ <includes>
+ <include>org.apache.hadoop:hadoop-core</include>
+ <include>commons-cli:commons-cli</include>
+ <include>commons-logging:commons-logging</include>
+ </includes>
+ <unpack>false</unpack>
+ <outputDirectory>.installer/hadoop-0.20.2/lib</outputDirectory>
+ </dependencySet>
+ <dependencySet>
+ <includes>
+ <include>org.apache.zookeeper:zookeeper</include>
+ <include>log4j:log4j</include>
+ <include>org.slf4j:slf4j-api</include>
+ </includes>
+ <unpack>false</unpack>
+ <outputDirectory>.installer/zookeeper/lib</outputDirectory>
+ </dependencySet>
+ <dependencySet>
+ <outputDirectory>asterix</outputDirectory>
+ <includes>
+ <include>asterix-server*</include>
+ </includes>
+ <useTransitiveDependencies>false</useTransitiveDependencies>
+ </dependencySet>
+ <dependencySet>
+ <outputDirectory>.installer/events</outputDirectory>
+ <includes>
+ <include>asterix-events*</include>
+ </includes>
+ <unpack>false</unpack>
+ <useTransitiveDependencies>false</useTransitiveDependencies>
+ </dependencySet>
+ </dependencySets>
+ </assembly>
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java
index 8e4b2bc..4be8cd5 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java
@@ -19,53 +19,60 @@
import org.kohsuke.args4j.Option;
import edu.uci.ics.asterix.common.configuration.AsterixConfiguration;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ILookupService;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
import edu.uci.ics.asterix.installer.driver.InstallerDriver;
import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.service.ILookupService;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
public class AlterCommand extends AbstractCommand {
- @Override
- protected void execCommand() throws Exception {
- InstallerDriver.initConfig(true);
- String instanceName = ((AlterConfig) config).name;
- InstallerUtil.validateAsterixInstanceExists(instanceName, State.INACTIVE);
- ILookupService lookupService = ServiceProvider.INSTANCE.getLookupService();
- AsterixInstance instance = lookupService.getAsterixInstance(instanceName);
- InstallerUtil.createClusterProperties(instance.getCluster(), instance.getAsterixConfiguration());
- AsterixConfiguration asterixConfiguration = InstallerUtil
- .getAsterixConfiguration(((AlterConfig) config).confPath);
- instance.setAsterixConfiguration(asterixConfiguration);
- instance.setModifiedTimestamp(new Date());
- lookupService.updateAsterixInstance(instance);
- LOGGER.info("Altered configuration settings for Asterix instance: " + instanceName);
+ @Override
+ protected void execCommand() throws Exception {
+ InstallerDriver.initConfig(true);
+ String instanceName = ((AlterConfig) config).name;
+ AsterixEventServiceUtil.validateAsterixInstanceExists(instanceName,
+ State.INACTIVE);
+ ILookupService lookupService = ServiceProvider.INSTANCE
+ .getLookupService();
+ AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService()
+ .getAsterixInstance(instanceName);
+ AsterixEventServiceUtil.createClusterProperties(instance.getCluster(),
+ instance.getAsterixConfiguration());
+ AsterixConfiguration asterixConfiguration = InstallerUtil
+ .getAsterixConfiguration(((AlterConfig) config).confPath);
+ instance.setAsterixConfiguration(asterixConfiguration);
+ instance.setModifiedTimestamp(new Date());
+ lookupService.updateAsterixInstance(instance);
+ LOGGER.info("Altered configuration settings for Asterix instance: "
+ + instanceName);
- }
+ }
- @Override
- protected CommandConfig getCommandConfig() {
- return new AlterConfig();
- }
+ @Override
+ protected CommandConfig getCommandConfig() {
+ return new AlterConfig();
+ }
- @Override
- protected String getUsageDescription() {
- return "\nAlter the instance's configuration settings."
- + "\nPrior to running this command, the instance is required to be INACTIVE state."
- + "\nChanged configuration settings will be reflected when the instance is started."
- + "\n\nAvailable arguments/options" + "\n-n name of the ASTERIX instance.";
- }
+ @Override
+ protected String getUsageDescription() {
+ return "\nAlter the instance's configuration settings."
+ + "\nPrior to running this command, the instance is required to be INACTIVE state."
+ + "\nChanged configuration settings will be reflected when the instance is started."
+ + "\n\nAvailable arguments/options"
+ + "\n-n name of the ASTERIX instance.";
+ }
}
class AlterConfig extends CommandConfig {
- @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
- public String name;
+ @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
+ public String name;
- @Option(name = "-a", required = true, usage = "Path to asterix instance configuration")
- public String confPath;
+ @Option(name = "-a", required = true, usage = "Path to asterix instance configuration")
+ public String confPath;
}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java
index 91cbc86..3f30619 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java
@@ -19,15 +19,16 @@
import org.kohsuke.args4j.Option;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.model.BackupInfo;
import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.events.PatternCreator;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.model.BackupInfo;
import edu.uci.ics.asterix.installer.schema.conf.Backup;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
public class BackupCommand extends AbstractCommand {
@@ -37,12 +38,12 @@
protected void execCommand() throws Exception {
InstallerDriver.initConfig(true);
String asterixInstanceName = ((BackupConfig) config).name;
- AsterixInstance instance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE);
+ AsterixInstance instance = AsterixEventServiceUtil.validateAsterixInstanceExists(asterixInstanceName,
+ State.INACTIVE);
List<BackupInfo> backupInfo = instance.getBackupInfo();
- PatternCreator pc = new PatternCreator();
- Backup backupConf = InstallerDriver.getConfiguration().getBackup();
- Patterns patterns = pc.getBackUpAsterixPattern(instance, backupConf);
- InstallerUtil.getEventrixClient(instance.getCluster()).submit(patterns);
+ Backup backupConf = AsterixEventService.getConfiguration().getBackup();
+ Patterns patterns = PatternCreator.INSTANCE.getBackUpAsterixPattern(instance, backupConf);
+ AsterixEventService.getAsterixEventServiceClient(instance.getCluster()).submit(patterns);
int backupId = backupInfo.size();
BackupInfo binfo = new BackupInfo(backupId, new Date(), backupConf);
backupInfo.add(binfo);
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandHandler.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandHandler.java
index 230a945..45cc9f7 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandHandler.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandHandler.java
@@ -52,6 +52,12 @@
case CONFIGURE:
cmd = new ConfigureCommand();
break;
+ case INSTALL:
+ cmd = new InstallCommand();
+ break;
+ case UNINSTALL:
+ cmd = new UninstallCommand();
+ break;
case LOG:
cmd = new LogCommand();
break;
@@ -61,6 +67,17 @@
case HELP:
cmd = new HelpCommand();
break;
+ case STOPNODE:
+ cmd = new StopNodeCommand();
+ break;
+ case STARTNODE:
+ cmd = new StartNodeCommand();
+ break;
+ case VERSION:
+ cmd = new VersionCommand();
+ break;
+ default:
+ break;
}
cmd.execute(args);
}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java
index 2803fee..63d2f33 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java
@@ -16,20 +16,25 @@
import java.io.File;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.Unmarshaller;
+
import org.kohsuke.args4j.Option;
import edu.uci.ics.asterix.common.configuration.AsterixConfiguration;
+import edu.uci.ics.asterix.event.error.VerificationUtil;
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
import edu.uci.ics.asterix.event.management.EventUtil;
-import edu.uci.ics.asterix.event.management.EventrixClient;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixRuntimeState;
import edu.uci.ics.asterix.event.schema.cluster.Cluster;
import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
import edu.uci.ics.asterix.installer.driver.InstallerDriver;
import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.error.VerificationUtil;
-import edu.uci.ics.asterix.installer.events.PatternCreator;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixRuntimeState;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
public class CreateCommand extends AbstractCommand {
@@ -46,29 +51,31 @@
throw new Exception("Cannot create an Asterix instance.");
}
asterixInstanceName = ((CreateConfig) config).name;
- InstallerUtil.validateAsterixInstanceNotExists(asterixInstanceName);
+ AsterixEventServiceUtil.validateAsterixInstanceNotExists(asterixInstanceName);
CreateConfig createConfig = (CreateConfig) config;
cluster = EventUtil.getCluster(createConfig.clusterPath);
+ cluster.setInstanceName(asterixInstanceName);
asterixConfiguration = InstallerUtil.getAsterixConfiguration(createConfig.asterixConfPath);
- AsterixInstance asterixInstance = InstallerUtil.createAsterixInstance(asterixInstanceName, cluster,
+ AsterixInstance asterixInstance = AsterixEventServiceUtil.createAsterixInstance(asterixInstanceName, cluster,
asterixConfiguration);
- InstallerUtil.evaluateConflictWithOtherInstances(asterixInstance);
- InstallerUtil.createAsterixZip(asterixInstance);
- InstallerUtil.createClusterProperties(cluster, asterixConfiguration);
- EventrixClient eventrixClient = InstallerUtil.getEventrixClient(cluster);
- PatternCreator pc = new PatternCreator();
+ AsterixEventServiceUtil.evaluateConflictWithOtherInstances(asterixInstance);
+ AsterixEventServiceUtil.createAsterixZip(asterixInstance);
+ AsterixEventServiceUtil.createClusterProperties(cluster, asterixConfiguration);
+ AsterixEventServiceClient eventrixClient = AsterixEventService.getAsterixEventServiceClient(cluster, true,
+ false);
- Patterns asterixBinarytrasnferPattern = pc.getAsterixBinaryTransferPattern(asterixInstanceName, cluster);
+ Patterns asterixBinarytrasnferPattern = PatternCreator.INSTANCE.getAsterixBinaryTransferPattern(
+ asterixInstanceName, cluster);
eventrixClient.submit(asterixBinarytrasnferPattern);
- Patterns patterns = pc.getStartAsterixPattern(asterixInstanceName, cluster);
+ Patterns patterns = PatternCreator.INSTANCE.getStartAsterixPattern(asterixInstanceName, cluster);
eventrixClient.submit(patterns);
AsterixRuntimeState runtimeState = VerificationUtil.getAsterixRuntimeState(asterixInstance);
VerificationUtil.updateInstanceWithRuntimeDescription(asterixInstance, runtimeState, true);
ServiceProvider.INSTANCE.getLookupService().writeAsterixInstance(asterixInstance);
- InstallerUtil.deleteDirectory(InstallerDriver.getManagixHome() + File.separator + InstallerDriver.ASTERIX_DIR
- + File.separator + asterixInstanceName);
+ AsterixEventServiceUtil.deleteDirectory(InstallerDriver.getManagixHome() + File.separator
+ + InstallerDriver.ASTERIX_DIR + File.separator + asterixInstanceName);
LOGGER.info(asterixInstance.getDescription(false));
}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java
index 00a3de4..3938ea5 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java
@@ -16,13 +16,14 @@
import org.kohsuke.args4j.Option;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.events.PatternCreator;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
public class DeleteCommand extends AbstractCommand {
@@ -30,13 +31,13 @@
protected void execCommand() throws Exception {
InstallerDriver.initConfig(true);
String asterixInstanceName = ((DeleteConfig) config).name;
- AsterixInstance instance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE);
- PatternCreator pc = new PatternCreator();
- Patterns patterns = pc.createDeleteInstancePattern(instance);
- InstallerUtil.getEventrixClient(instance.getCluster()).submit(patterns);
+ AsterixInstance instance = AsterixEventServiceUtil.validateAsterixInstanceExists(asterixInstanceName,
+ State.INACTIVE);
+ Patterns patterns = PatternCreator.INSTANCE.createDeleteInstancePattern(instance);
+ AsterixEventService.getAsterixEventServiceClient(instance.getCluster()).submit(patterns);
- patterns = pc.createRemoveAsterixWorkingDirPattern(instance);
- InstallerUtil.getEventrixClient(instance.getCluster()).submit(patterns);
+ patterns = PatternCreator.INSTANCE.createRemoveAsterixWorkingDirPattern(instance);
+ AsterixEventService.getAsterixEventServiceClient(instance.getCluster()).submit(patterns);
ServiceProvider.INSTANCE.getLookupService().removeAsterixInstance(asterixInstanceName);
LOGGER.info("Deleted Asterix instance: " + asterixInstanceName);
}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java
index 58851cb..9010061 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java
@@ -18,14 +18,14 @@
import org.kohsuke.args4j.Option;
+import edu.uci.ics.asterix.event.error.VerificationUtil;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.model.AsterixRuntimeState;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
import edu.uci.ics.asterix.installer.error.InstallerException;
-import edu.uci.ics.asterix.installer.error.VerificationUtil;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.model.AsterixRuntimeState;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
public class DescribeCommand extends AbstractCommand {
@@ -35,7 +35,7 @@
String asterixInstanceName = ((DescribeConfig) config).name;
boolean adminView = ((DescribeConfig) config).admin;
if (asterixInstanceName != null) {
- InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE, State.ACTIVE,
+ AsterixEventServiceUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE, State.ACTIVE,
State.UNUSABLE);
AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService().getAsterixInstance(
asterixInstanceName);
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/HelpCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/HelpCommand.java
index a2f00b3..68f8532 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/HelpCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/HelpCommand.java
@@ -55,12 +55,26 @@
case VALIDATE:
helpMessage = new ValidateCommand().getUsageDescription();
break;
+ case INSTALL:
+ helpMessage = new InstallCommand().getUsageDescription();
+ break;
+ case UNINSTALL:
+ helpMessage = new UninstallCommand().getUsageDescription();
case ALTER:
helpMessage = new AlterCommand().getUsageDescription();
break;
case LOG:
helpMessage = new LogCommand().getUsageDescription();
break;
+ case STOPNODE:
+ helpMessage = new StopNodeCommand().getUsageDescription();
+ break;
+ case STARTNODE:
+ helpMessage = new StartNodeCommand().getUsageDescription();
+ break;
+ case VERSION:
+ helpMessage = new VersionCommand().getUsageDescription();
+ break;
default:
helpMessage = "Unknown command " + command;
}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ICommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ICommand.java
index 9e67bf5..288c882 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ICommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ICommand.java
@@ -27,9 +27,14 @@
ALTER,
VALIDATE,
CONFIGURE,
+ INSTALL,
+ UNINSTALL,
LOG,
SHUTDOWN,
- HELP
+ HELP,
+ STOPNODE,
+ STARTNODE,
+ VERSION
}
public void execute(String args[]) throws Exception;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/InstallCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/InstallCommand.java
new file mode 100644
index 0000000..59a69bf
--- /dev/null
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/InstallCommand.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.installer.command;
+
+import org.kohsuke.args4j.Option;
+
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.util.PatternCreator;
+import edu.uci.ics.asterix.installer.driver.InstallerDriver;
+
+public class InstallCommand extends AbstractCommand {
+
+ @Override
+ protected void execCommand() throws Exception {
+ InstallerDriver.initConfig(true);
+ InstallConfig installConfig = ((InstallConfig) config);
+ String instanceName = installConfig.name;
+ AsterixInstance instance = AsterixEventServiceUtil.validateAsterixInstanceExists(instanceName, State.INACTIVE);
+ PatternCreator pc = PatternCreator.INSTANCE;
+ Patterns patterns = pc.getLibraryInstallPattern(instance, installConfig.dataverseName,
+ installConfig.libraryName, installConfig.libraryPath);
+ AsterixEventService.getAsterixEventServiceClient(instance.getCluster()).submit(patterns);
+ LOGGER.info("Installed library " + installConfig.libraryName);
+ }
+
+ @Override
+ protected CommandConfig getCommandConfig() {
+ return new InstallConfig();
+ }
+
+ @Override
+ protected String getUsageDescription() {
+ return "Installs a library to an asterix instance." + "\n" + "Arguments/Options\n"
+ + "-n Name of Asterix Instance\n"
+ + "-d Name of the dataverse under which the library will be installed\n" + "-l Name of the library\n"
+ + "-p Path to library zip bundle";
+
+ }
+
+}
+
+class InstallConfig extends CommandConfig {
+
+ @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
+ public String name;
+
+ @Option(name = "-d", required = true, usage = "Name of the dataverse under which the library will be installed")
+ public String dataverseName;
+
+ @Option(name = "-l", required = true, usage = "Name of the library")
+ public String libraryName;
+
+ @Option(name = "-p", required = true, usage = "Path to library zip bundle")
+ public String libraryPath;
+
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/LogCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/LogCommand.java
index 11b4aa7..bf30a48 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/LogCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/LogCommand.java
@@ -21,14 +21,15 @@
import org.apache.commons.io.FileUtils;
import org.kohsuke.args4j.Option;
-import edu.uci.ics.asterix.event.management.EventrixClient;
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.util.PatternCreator;
import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
import edu.uci.ics.asterix.installer.error.InstallerException;
-import edu.uci.ics.asterix.installer.events.PatternCreator;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
public class LogCommand extends AbstractCommand {
@@ -36,12 +37,15 @@
protected void execCommand() throws Exception {
InstallerDriver.initConfig(true);
String asterixInstanceName = ((LogConfig) config).name;
- AsterixInstance instance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE,
- State.UNUSABLE, State.ACTIVE);
- PatternCreator pc = new PatternCreator();
- EventrixClient client = InstallerUtil.getEventrixClient(instance.getCluster());
- String outputDir = ((LogConfig) config).outputDir == null ? InstallerDriver.getManagixHome() + File.separator + "logdump"
- : ((LogConfig) config).outputDir;
+ AsterixInstance instance = AsterixEventServiceUtil.validateAsterixInstanceExists(asterixInstanceName,
+ State.INACTIVE, State.UNUSABLE, State.ACTIVE);
+ PatternCreator pc = PatternCreator.INSTANCE;
+
+ AsterixEventServiceClient eventrixClient = AsterixEventService.getAsterixEventServiceClient(
+ instance.getCluster(), true, false);
+
+ String outputDir = ((LogConfig) config).outputDir == null ? InstallerDriver.getManagixHome() + File.separator
+ + "logdump" : ((LogConfig) config).outputDir;
File f = new File(outputDir);
String outputDirPath = f.getAbsolutePath();
if (!f.exists()) {
@@ -50,12 +54,13 @@
throw new InstallerException("Unable to create output directory:" + outputDirPath);
}
}
- Patterns transferLogPattern = pc.getGenerateLogPattern(asterixInstanceName, instance.getCluster(), outputDirPath);
- client.submit(transferLogPattern);
+ Patterns transferLogPattern = pc.getGenerateLogPattern(asterixInstanceName, instance.getCluster(),
+ outputDirPath);
+ eventrixClient.submit(transferLogPattern);
File outputDirFile = new File(outputDirPath);
final String destFileName = "log_" + new Date().toString().replace(' ', '_') + ".zip";
File destFile = new File(outputDirFile, destFileName);
- InstallerUtil.zipDir(outputDirFile, destFile);
+ AsterixEventServiceUtil.zipDir(outputDirFile, destFile);
String[] filesToDelete = outputDirFile.list(new FilenameFilter() {
@Override
@@ -65,7 +70,7 @@
});
for (String fileS : filesToDelete) {
- f = new File(outputDirFile, fileS);
+ f = new File(outputDirFile, fileS);
if (f.isDirectory()) {
FileUtils.deleteDirectory(f);
} else {
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java
index 6627ac2..5b76099 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java
@@ -18,13 +18,14 @@
import org.kohsuke.args4j.Option;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.model.BackupInfo;
import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.util.PatternCreator;
import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.events.PatternCreator;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.model.BackupInfo;
public class RestoreCommand extends AbstractCommand {
@@ -32,7 +33,8 @@
protected void execCommand() throws Exception {
InstallerDriver.initConfig(true);
String asterixInstanceName = ((RestoreConfig) config).name;
- AsterixInstance instance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE);
+ AsterixInstance instance = AsterixEventServiceUtil.validateAsterixInstanceExists(asterixInstanceName,
+ State.INACTIVE);
int backupId = ((RestoreConfig) config).backupId;
List<BackupInfo> backupInfoList = instance.getBackupInfo();
if (backupInfoList.size() <= backupId || backupId < 0) {
@@ -40,9 +42,8 @@
}
BackupInfo backupInfo = backupInfoList.get(backupId);
- PatternCreator pc = new PatternCreator();
- Patterns patterns = pc.getRestoreAsterixPattern(instance, backupInfo);
- InstallerUtil.getEventrixClient(instance.getCluster()).submit(patterns);
+ Patterns patterns = PatternCreator.INSTANCE.getRestoreAsterixPattern(instance, backupInfo);
+ AsterixEventService.getAsterixEventServiceClient(instance.getCluster()).submit(patterns);
LOGGER.info("Asterix instance: " + asterixInstanceName + " has been restored from backup");
}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java
index 0c44bd8..228cb1c 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java
@@ -14,9 +14,10 @@
*/
package edu.uci.ics.asterix.installer.command;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.ILookupService;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.service.ILookupService;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
public class ShutdownCommand extends AbstractCommand {
@@ -24,7 +25,7 @@
protected void execCommand() throws Exception {
InstallerDriver.initConfig(false);
ILookupService lookupService = ServiceProvider.INSTANCE.getLookupService();
- lookupService.stopService(InstallerDriver.getConfiguration());
+ lookupService.stopService(AsterixEventService.getConfiguration());
}
@Override
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java
index 3fc7e96..63ec6fc 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java
@@ -18,16 +18,17 @@
import org.kohsuke.args4j.Option;
-import edu.uci.ics.asterix.event.management.EventrixClient;
+import edu.uci.ics.asterix.event.error.VerificationUtil;
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.model.AsterixRuntimeState;
import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.error.VerificationUtil;
-import edu.uci.ics.asterix.installer.events.PatternCreator;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixRuntimeState;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
public class StartCommand extends AbstractCommand {
@@ -35,18 +36,18 @@
protected void execCommand() throws Exception {
InstallerDriver.initConfig(true);
String asterixInstanceName = ((StartConfig) config).name;
- AsterixInstance instance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE);
- InstallerUtil.createAsterixZip(instance);
- PatternCreator pc = new PatternCreator();
- EventrixClient client = InstallerUtil.getEventrixClient(instance.getCluster());
- Patterns asterixBinaryTransferPattern = pc.getAsterixBinaryTransferPattern(asterixInstanceName,
- instance.getCluster());
+ AsterixInstance instance = AsterixEventServiceUtil.validateAsterixInstanceExists(asterixInstanceName,
+ State.INACTIVE);
+ AsterixEventServiceUtil.createAsterixZip(instance);
+ AsterixEventServiceClient client = AsterixEventService.getAsterixEventServiceClient(instance.getCluster());
+ Patterns asterixBinaryTransferPattern = PatternCreator.INSTANCE.getAsterixBinaryTransferPattern(
+ asterixInstanceName, instance.getCluster());
client.submit(asterixBinaryTransferPattern);
- InstallerUtil.createClusterProperties(instance.getCluster(), instance.getAsterixConfiguration());
- Patterns patterns = pc.getStartAsterixPattern(asterixInstanceName, instance.getCluster());
+ AsterixEventServiceUtil.createClusterProperties(instance.getCluster(), instance.getAsterixConfiguration());
+ Patterns patterns = PatternCreator.INSTANCE.getStartAsterixPattern(asterixInstanceName, instance.getCluster());
client.submit(patterns);
- InstallerUtil.deleteDirectory(InstallerDriver.getManagixHome() + File.separator + InstallerDriver.ASTERIX_DIR
- + File.separator + asterixInstanceName);
+ AsterixEventServiceUtil.deleteDirectory(InstallerDriver.getManagixHome() + File.separator
+ + InstallerDriver.ASTERIX_DIR + File.separator + asterixInstanceName);
AsterixRuntimeState runtimeState = VerificationUtil.getAsterixRuntimeState(instance);
VerificationUtil.updateInstanceWithRuntimeDescription(instance, runtimeState, true);
LOGGER.info(instance.getDescription(false));
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartNodeCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartNodeCommand.java
new file mode 100644
index 0000000..7d0a6ee
--- /dev/null
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartNodeCommand.java
@@ -0,0 +1,101 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.installer.command;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.kohsuke.args4j.Option;
+
+import edu.uci.ics.asterix.event.error.VerificationUtil;
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.model.AsterixRuntimeState;
+import edu.uci.ics.asterix.event.model.ProcessInfo;
+import edu.uci.ics.asterix.event.schema.cluster.Cluster;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
+import edu.uci.ics.asterix.event.schema.pattern.Pattern;
+import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
+import edu.uci.ics.asterix.installer.driver.InstallerDriver;
+import edu.uci.ics.asterix.installer.error.InstallerException;
+
+public class StartNodeCommand extends AbstractCommand {
+
+ @Override
+ protected void execCommand() throws Exception {
+ InstallerDriver.initConfig(true);
+ String asterixInstanceName = ((StartNodeConfig) config).name;
+ AsterixInstance instance = AsterixEventServiceUtil.validateAsterixInstanceExists(asterixInstanceName,
+ State.INACTIVE, State.ACTIVE, State.UNUSABLE);
+
+ Cluster cluster = instance.getCluster();
+ List<Pattern> pl = new ArrayList<Pattern>();
+ AsterixRuntimeState runtimeState = VerificationUtil.getAsterixRuntimeState(instance);
+ String[] nodesToBeAdded = ((StartNodeConfig) config).nodes.split(",");
+ List<String> aliveNodes = new ArrayList<String>();
+ for (ProcessInfo p : runtimeState.getProcesses()) {
+ aliveNodes.add(p.getNodeId());
+ }
+ List<Node> clusterNodes = cluster.getNode();
+ for (String n : nodesToBeAdded) {
+ if (aliveNodes.contains(n)) {
+ throw new InstallerException("Node: " + n + " is already alive");
+ }
+ for (Node node : clusterNodes) {
+ if (n.equals(node.getId())) {
+ String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
+ Pattern createNC = PatternCreator.INSTANCE.createNCStartPattern(cluster.getMasterNode()
+ .getClusterIp(), node.getId(), asterixInstanceName + "_" + node.getId(), iodevices);
+ pl.add(createNC);
+ break;
+ }
+ }
+ }
+ Patterns patterns = new Patterns(pl);
+ AsterixEventServiceClient client = AsterixEventService.getAsterixEventServiceClient(cluster);
+ client.submit(patterns);
+ runtimeState = VerificationUtil.getAsterixRuntimeState(instance);
+ VerificationUtil.updateInstanceWithRuntimeDescription(instance, runtimeState, true);
+ LOGGER.info(instance.getDescription(false));
+ ServiceProvider.INSTANCE.getLookupService().updateAsterixInstance(instance);
+ }
+
+ @Override
+ protected CommandConfig getCommandConfig() {
+ return new StartNodeConfig();
+ }
+
+ @Override
+ protected String getUsageDescription() {
+ return "\nStarts a set of nodes for an ASTERIX instance." + "\n\nAvailable arguments/options"
+ + "\n-n name of the ASTERIX instance. " + "\n-nodes"
+ + "Comma separated list of nodes that need to be started";
+ }
+}
+
+class StartNodeConfig extends CommandConfig {
+
+ @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
+ public String name;
+
+ @Option(name = "-nodes", required = true, usage = "Comma separated list of nodes that need to be started")
+ public String nodes;
+
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java
index c00fa86..820948c 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java
@@ -20,16 +20,17 @@
import org.kohsuke.args4j.Option;
-import edu.uci.ics.asterix.event.management.EventrixClient;
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
import edu.uci.ics.asterix.event.schema.cluster.Node;
import edu.uci.ics.asterix.event.schema.pattern.Pattern;
import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.events.PatternCreator;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
public class StopCommand extends AbstractCommand {
@@ -37,18 +38,19 @@
protected void execCommand() throws Exception {
InstallerDriver.initConfig(true);
String asterixInstanceName = ((StopConfig) config).name;
- AsterixInstance asterixInstance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName,
+ AsterixInstance asterixInstance = AsterixEventServiceUtil.validateAsterixInstanceExists(asterixInstanceName,
State.ACTIVE, State.UNUSABLE);
- PatternCreator pc = new PatternCreator();
- EventrixClient client = InstallerUtil.getEventrixClient(asterixInstance.getCluster());
+ AsterixEventServiceClient client = AsterixEventService.getAsterixEventServiceClient(asterixInstance.getCluster());
List<Pattern> ncKillPatterns = new ArrayList<Pattern>();
for (Node node : asterixInstance.getCluster().getNode()) {
- ncKillPatterns.add(pc.createNCStopPattern(node.getId(), asterixInstanceName + "_" + node.getId()));
+ ncKillPatterns.add(PatternCreator.INSTANCE.createNCStopPattern(node.getId(), asterixInstanceName + "_"
+ + node.getId()));
}
List<Pattern> ccKillPatterns = new ArrayList<Pattern>();
- ccKillPatterns.add(pc.createCCStopPattern(asterixInstance.getCluster().getMasterNode().getId()));
+ ccKillPatterns.add(PatternCreator.INSTANCE.createCCStopPattern(asterixInstance.getCluster().getMasterNode()
+ .getId()));
try {
client.submit(new Patterns(ncKillPatterns));
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopNodeCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopNodeCommand.java
new file mode 100644
index 0000000..c7cc6de
--- /dev/null
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopNodeCommand.java
@@ -0,0 +1,108 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.installer.command;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+
+import org.kohsuke.args4j.Option;
+
+import edu.uci.ics.asterix.event.error.VerificationUtil;
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.model.AsterixRuntimeState;
+import edu.uci.ics.asterix.event.model.ProcessInfo;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
+import edu.uci.ics.asterix.event.schema.pattern.Pattern;
+import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
+import edu.uci.ics.asterix.installer.driver.InstallerDriver;
+import edu.uci.ics.asterix.installer.error.InstallerException;
+
+public class StopNodeCommand extends AbstractCommand {
+
+ @Override
+ protected void execCommand() throws Exception {
+ InstallerDriver.initConfig(true);
+ String asterixInstanceName = ((StopNodeConfig) config).name;
+ AsterixInstance asterixInstance = AsterixEventServiceUtil.validateAsterixInstanceExists(asterixInstanceName,
+ State.ACTIVE, State.UNUSABLE);
+
+ AsterixEventServiceClient client = AsterixEventService.getAsterixEventServiceClient(asterixInstance
+ .getCluster());
+
+ String[] nodesToStop = ((StopNodeConfig) config).nodeList.split(",");
+ AsterixRuntimeState runtimeState = VerificationUtil.getAsterixRuntimeState(asterixInstance);
+ List<String> aliveNodes = new ArrayList<String>();
+ for (ProcessInfo p : runtimeState.getProcesses()) {
+ aliveNodes.add(p.getNodeId());
+ }
+
+ List<String> validNodeIds = new ArrayList<String>();
+ for (Node node : asterixInstance.getCluster().getNode()) {
+ validNodeIds.add(node.getId());
+ }
+ List<Pattern> ncKillPatterns = new ArrayList<Pattern>();
+ for (String nodeId : nodesToStop) {
+ if (!nodeId.contains(nodeId)) {
+ throw new InstallerException("Invalid nodeId: " + nodeId);
+ }
+ if (!aliveNodes.contains(nodeId)) {
+ throw new InstallerException("Node: " + nodeId + " is not alive");
+ }
+ ncKillPatterns.add(PatternCreator.INSTANCE.createNCStopPattern(nodeId, asterixInstanceName + "_" + nodeId));
+ }
+
+ try {
+ client.submit(new Patterns(ncKillPatterns));
+ } catch (Exception e) {
+ // processes are already dead
+ LOGGER.debug("Attempt to kill non-existing processess");
+ }
+
+ asterixInstance.setStateChangeTimestamp(new Date());
+ ServiceProvider.INSTANCE.getLookupService().updateAsterixInstance(asterixInstance);
+ LOGGER.info("Stopped nodes " + ((StopNodeConfig) config).nodeList + " serving Asterix instance: "
+ + asterixInstanceName);
+ }
+
+ @Override
+ protected CommandConfig getCommandConfig() {
+ return new StopNodeConfig();
+ }
+
+ @Override
+ protected String getUsageDescription() {
+ return "\nStops a specified set of ASTERIX nodes." + "\n\nAvailable arguments/options"
+ + "\n-n name of the ASTERIX instance. "
+ + "\n-nodes Comma separated list of nodes that need to be stopped. ";
+
+ }
+}
+
+class StopNodeConfig extends CommandConfig {
+
+ @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
+ public String name;
+
+ @Option(name = "-nodes", required = true, usage = "Comma separated list of nodes that need to be stopped")
+ public String nodeList;
+
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/UninstallCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/UninstallCommand.java
new file mode 100644
index 0000000..39872e7
--- /dev/null
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/UninstallCommand.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.installer.command;
+
+import org.kohsuke.args4j.Option;
+
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ILookupService;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
+import edu.uci.ics.asterix.installer.driver.InstallerDriver;
+
+public class UninstallCommand extends AbstractCommand {
+
+ @Override
+ protected void execCommand() throws Exception {
+ InstallerDriver.initConfig(true);
+ UninstallConfig uninstallConfig = ((UninstallConfig) config);
+ String instanceName = uninstallConfig.name;
+ AsterixEventServiceUtil.validateAsterixInstanceExists(instanceName, State.INACTIVE);
+ ILookupService lookupService = ServiceProvider.INSTANCE.getLookupService();
+ AsterixInstance instance = lookupService.getAsterixInstance(instanceName);
+ PatternCreator pc = PatternCreator.INSTANCE;
+ Patterns patterns = pc.getLibraryUninstallPattern(instance, uninstallConfig.dataverseName,
+ uninstallConfig.libraryName);
+ AsterixEventServiceClient client = AsterixEventService.getAsterixEventServiceClient(instance.getCluster());
+ client.submit(patterns);
+ LOGGER.info("Uninstalled library " + uninstallConfig.libraryName);
+ }
+
+ @Override
+ protected CommandConfig getCommandConfig() {
+ return new UninstallConfig();
+ }
+
+ @Override
+ protected String getUsageDescription() {
+ return "Uninstalls a library from an asterix instance." + "\n" + "Arguments/Options\n"
+ + "-n Name of Asterix Instance\n"
+ + "-d Name of the dataverse under which the library will be installed\n" + "-l Name of the library\n"
+ + "-l Name of the library";
+ }
+
+}
+
+class UninstallConfig extends CommandConfig {
+
+ @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
+ public String name;
+
+ @Option(name = "-d", required = true, usage = "Name of the dataverse under which the library will be installed")
+ public String dataverseName;
+
+ @Option(name = "-l", required = true, usage = "Name of the library")
+ public String libraryName;
+
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ValidateCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ValidateCommand.java
index 5b63166..5035028 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ValidateCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ValidateCommand.java
@@ -29,8 +29,8 @@
import edu.uci.ics.asterix.event.schema.cluster.Cluster;
import edu.uci.ics.asterix.event.schema.cluster.MasterNode;
import edu.uci.ics.asterix.event.schema.cluster.Node;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
import edu.uci.ics.asterix.installer.schema.conf.Configuration;
import edu.uci.ics.asterix.installer.schema.conf.Zookeeper;
@@ -74,17 +74,10 @@
public boolean validateEnvironment() throws Exception {
boolean valid = true;
- String managixHome = System.getenv(InstallerDriver.ENV_MANAGIX_HOME);
- if (managixHome == null) {
+ File home = new File(InstallerDriver.getManagixHome());
+ if (!home.exists()) {
valid = false;
- LOGGER.fatal(InstallerDriver.ENV_MANAGIX_HOME + " not set " + ERROR);
- } else {
- File home = new File(managixHome);
- if (!home.exists()) {
- valid = false;
- LOGGER.fatal(InstallerDriver.ENV_MANAGIX_HOME + ": " + home.getAbsolutePath() + " does not exist!"
- + ERROR);
- }
+ LOGGER.fatal(InstallerDriver.ENV_MANAGIX_HOME + ": " + home.getAbsolutePath() + " does not exist!" + ERROR);
}
return valid;
@@ -109,7 +102,7 @@
MasterNode masterNode = cluster.getMasterNode();
Node master = new Node(masterNode.getId(), masterNode.getClusterIp(), masterNode.getJavaHome(),
- masterNode.getLogDir(), null, null, null);
+ masterNode.getLogDir(), null, null, null, null);
ipAddresses.add(masterNode.getClusterIp());
valid = valid & validateNodeConfiguration(master, cluster);
@@ -143,7 +136,7 @@
+ File.separator + "scripts" + File.separator + "validate_ssh.sh";
List<String> args = ipAddresses;
args.add(0, username);
- String output = InstallerUtil.executeLocalScript(script, args);
+ String output = AsterixEventServiceUtil.executeLocalScript(script, args);
ipAddresses.remove(0);
for (String line : output.split("\n")) {
ipAddresses.remove(line);
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/VersionCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/VersionCommand.java
new file mode 100644
index 0000000..041839d
--- /dev/null
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/VersionCommand.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.installer.command;
+
+import java.io.File;
+
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.installer.driver.InstallerDriver;
+
+public class VersionCommand extends AbstractCommand {
+
+ @Override
+ protected void execCommand() throws Exception {
+ InstallerDriver.initConfig(false);
+ String asterixZipName = AsterixEventService.getAsterixZip().substring(
+ AsterixEventService.getAsterixZip().lastIndexOf(File.separator) + 1);
+ String asterixVersion = asterixZipName.substring("asterix-server-".length(),
+ asterixZipName.indexOf("-binary-assembly"));
+ LOGGER.info("Asterix/Managix version " + asterixVersion);
+ }
+
+ @Override
+ protected CommandConfig getCommandConfig() {
+ return new VersionConfig();
+ }
+
+ @Override
+ protected String getUsageDescription() {
+ return "Provides version of Managix/Asterix";
+ }
+
+}
+
+class VersionConfig extends CommandConfig {
+
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java
index 521adc6..53ec136 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java
@@ -15,52 +15,38 @@
package edu.uci.ics.asterix.installer.driver;
import java.io.File;
-import java.io.FileFilter;
import javax.xml.bind.JAXBContext;
import javax.xml.bind.Unmarshaller;
-import org.apache.log4j.Level;
import org.apache.log4j.Logger;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.ILookupService;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
import edu.uci.ics.asterix.installer.command.CommandHandler;
import edu.uci.ics.asterix.installer.schema.conf.Configuration;
-import edu.uci.ics.asterix.installer.service.ILookupService;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
public class InstallerDriver {
- public static final String MANAGIX_INTERNAL_DIR = ".installer";
- public static final String MANAGIX_EVENT_DIR = MANAGIX_INTERNAL_DIR + File.separator + "eventrix";
- public static final String MANAGIX_EVENT_SCRIPTS_DIR = MANAGIX_INTERNAL_DIR + File.separator + "eventrix"
- + File.separator + "scripts";
- public static final String DEFAULT_ASTERIX_CONFIGURATION_PATH = "conf" + File.separator + File.separator
- + "asterix-configuration.xml";
- public static final String ASTERIX_DIR = "asterix";
- public static final String EVENTS_DIR = "events";
-
private static final Logger LOGGER = Logger.getLogger(InstallerDriver.class.getName());
+
+ public static final String MANAGIX_INTERNAL_DIR = ".installer";
public static final String ENV_MANAGIX_HOME = "MANAGIX_HOME";
public static final String MANAGIX_CONF_XML = "conf" + File.separator + "managix-conf.xml";
+ public static final String ASTERIX_DIR = "asterix";
- private static Configuration conf;
private static String managixHome;
- private static String asterixZip;
-
- public static String getAsterixZip() {
- return asterixZip;
- }
-
- public static Configuration getConfiguration() {
- return conf;
- }
public static void initConfig(boolean ensureLookupServiceIsRunning) throws Exception {
File configFile = new File(managixHome + File.separator + MANAGIX_CONF_XML);
JAXBContext configCtx = JAXBContext.newInstance(Configuration.class);
Unmarshaller unmarshaller = configCtx.createUnmarshaller();
- conf = (Configuration) unmarshaller.unmarshal(configFile);
- asterixZip = initBinary("asterix-server");
+ Configuration conf = (Configuration) unmarshaller.unmarshal(configFile);
+ String asterixDir = managixHome + File.separator + ASTERIX_DIR;
+ String eventHome = managixHome + File.separator + MANAGIX_INTERNAL_DIR;
+ AsterixEventService.initialize(conf, asterixDir, eventHome);
ILookupService lookupService = ServiceProvider.INSTANCE.getLookupService();
if (ensureLookupServiceIsRunning && !lookupService.isRunning(conf)) {
@@ -68,28 +54,6 @@
}
}
- private static String initBinary(final String fileNamePattern) {
- String asterixDir = InstallerDriver.getAsterixDir();
- File file = new File(asterixDir);
- File[] zipFiles = file.listFiles(new FileFilter() {
- public boolean accept(File arg0) {
- return arg0.getAbsolutePath().contains(fileNamePattern) && arg0.isFile();
- }
- });
- if (zipFiles.length == 0) {
- String msg = " Binary not found at " + asterixDir;
- LOGGER.log(Level.FATAL, msg);
- throw new IllegalStateException(msg);
- }
- if (zipFiles.length > 1) {
- String msg = " Multiple binaries found at " + asterixDir;
- LOGGER.log(Level.FATAL, msg);
- throw new IllegalStateException(msg);
- }
-
- return zipFiles[0].getAbsolutePath();
- }
-
public static String getManagixHome() {
return managixHome;
}
@@ -98,10 +62,6 @@
InstallerDriver.managixHome = managixHome;
}
- public static String getAsterixDir() {
- return managixHome + File.separator + ASTERIX_DIR;
- }
-
public static void main(String args[]) {
try {
if (args.length != 0) {
@@ -134,12 +94,18 @@
buffer.append("alter " + ":" + " Alter the instance's configuration settings" + "\n");
buffer.append("describe " + ":" + " Describes an existing asterix instance" + "\n");
buffer.append("validate " + ":" + " Validates the installer/cluster configuration" + "\n");
- buffer.append("configure" + ":" + " Configure the Asterix installer" + "\n");
+ buffer.append("configure" + ":" + " Auto-generate configuration for local psedu-distributed Asterix instance"
+ + "\n");
+ buffer.append("install " + ":" + " Installs a library to an asterix instance" + "\n");
+ buffer.append("uninstall" + ":" + " Uninstalls a library from an asterix instance" + "\n");
buffer.append("log " + ":"
+ " Produce a tar archive contianing log files from the master and worker nodes" + "\n");
buffer.append("shutdown " + ":" + " Shutdown the installer service" + "\n");
buffer.append("help " + ":" + " Provides usage description of a command" + "\n");
+ buffer.append("version " + ":" + " Provides version of Asterix/Managix" + "\n");
+
buffer.append("\nTo get more information about a command, use managix help -cmd <command>");
LOGGER.info(buffer.toString());
}
+
}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
index 2e04bbf..2569363 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
@@ -14,58 +14,23 @@
*/
package edu.uci.ics.asterix.installer.driver;
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
import java.io.File;
-import java.io.FileFilter;
-import java.io.FileInputStream;
import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.FileWriter;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.StringWriter;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.ArrayList;
-import java.util.Enumeration;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Random;
-import java.util.jar.JarEntry;
-import java.util.jar.JarFile;
-import java.util.jar.JarOutputStream;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipInputStream;
-import java.util.zip.ZipOutputStream;
import javax.xml.bind.JAXBContext;
import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
import javax.xml.bind.Unmarshaller;
-import org.apache.commons.io.IOUtils;
-
import edu.uci.ics.asterix.common.configuration.AsterixConfiguration;
-import edu.uci.ics.asterix.common.configuration.Coredump;
-import edu.uci.ics.asterix.common.configuration.Store;
-import edu.uci.ics.asterix.common.configuration.TransactionLogDir;
-import edu.uci.ics.asterix.event.driver.EventDriver;
-import edu.uci.ics.asterix.event.management.EventUtil;
-import edu.uci.ics.asterix.event.management.EventrixClient;
import edu.uci.ics.asterix.event.schema.cluster.Cluster;
-import edu.uci.ics.asterix.event.schema.cluster.Env;
import edu.uci.ics.asterix.event.schema.cluster.Node;
-import edu.uci.ics.asterix.event.schema.cluster.Property;
-import edu.uci.ics.asterix.installer.error.InstallerException;
-import edu.uci.ics.asterix.installer.error.OutputHandler;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
public class InstallerUtil {
+ private static final String DEFAULT_ASTERIX_CONFIGURATION_PATH = "conf" + File.separator
+ + "asterix-configuration.xml";
+
public static final String TXN_LOG_DIR = "txnLogs";
public static final String TXN_LOG_DIR_KEY_SUFFIX = "txnLogDir";
public static final String ASTERIX_CONFIGURATION_FILE = "asterix-configuration.xml";
@@ -75,133 +40,6 @@
public static final int HTTP_PORT_DEFAULT = 8888;
public static final int WEB_INTERFACE_PORT_DEFAULT = 19001;
- public static AsterixInstance createAsterixInstance(String asterixInstanceName, Cluster cluster,
- AsterixConfiguration asterixConfiguration) throws FileNotFoundException, IOException {
- Node metadataNode = getMetadataNode(cluster);
- String asterixZipName = InstallerDriver.getAsterixZip().substring(
- InstallerDriver.getAsterixZip().lastIndexOf(File.separator) + 1);
- String asterixVersion = asterixZipName.substring("asterix-server-".length(),
- asterixZipName.indexOf("-binary-assembly"));
- AsterixInstance instance = new AsterixInstance(asterixInstanceName, cluster, asterixConfiguration,
- metadataNode.getId(), asterixVersion);
- return instance;
- }
-
- public static void createAsterixZip(AsterixInstance asterixInstance) throws IOException, InterruptedException,
- JAXBException, InstallerException {
-
- String modifiedZipPath = injectAsterixPropertyFile(InstallerDriver.getAsterixZip(), asterixInstance);
- injectAsterixLogPropertyFile(modifiedZipPath, asterixInstance);
- }
-
- public static void createClusterProperties(Cluster cluster, AsterixConfiguration asterixConfiguration) {
- List<Property> clusterProperties = null;
- if (cluster.getEnv() != null && cluster.getEnv().getProperty() != null) {
- clusterProperties = cluster.getEnv().getProperty();
- clusterProperties.clear();
- } else {
- clusterProperties = new ArrayList<Property>();
- }
- for (edu.uci.ics.asterix.common.configuration.Property property : asterixConfiguration.getProperty()) {
- if (property.getName().equalsIgnoreCase(EventUtil.CC_JAVA_OPTS)) {
- clusterProperties.add(new Property(EventUtil.CC_JAVA_OPTS, property.getValue()));
- } else if (property.getName().equalsIgnoreCase(EventUtil.NC_JAVA_OPTS)) {
- clusterProperties.add(new Property(EventUtil.NC_JAVA_OPTS, property.getValue()));
- }
- }
- clusterProperties.add(new Property("ASTERIX_HOME", cluster.getWorkingDir().getDir() + File.separator
- + "asterix"));
- clusterProperties.add(new Property("LOG_DIR", cluster.getLogDir()));
- clusterProperties.add(new Property("JAVA_HOME", cluster.getJavaHome()));
- clusterProperties.add(new Property("WORKING_DIR", cluster.getWorkingDir().getDir()));
- clusterProperties.add(new Property("CLIENT_NET_IP", cluster.getMasterNode().getClientIp()));
- clusterProperties.add(new Property("CLUSTER_NET_IP", cluster.getMasterNode().getClusterIp()));
-
- int clusterNetPort = cluster.getMasterNode().getClusterPort() != null ? cluster.getMasterNode()
- .getClusterPort().intValue() : CLUSTER_NET_PORT_DEFAULT;
- int clientNetPort = cluster.getMasterNode().getClientPort() != null ? cluster.getMasterNode().getClientPort()
- .intValue() : CLIENT_NET_PORT_DEFAULT;
- int httpPort = cluster.getMasterNode().getHttpPort() != null ? cluster.getMasterNode().getHttpPort().intValue()
- : HTTP_PORT_DEFAULT;
-
- clusterProperties.add(new Property("CLIENT_NET_PORT", "" + clientNetPort));
- clusterProperties.add(new Property("CLUSTER_NET_PORT", "" + clusterNetPort));
- clusterProperties.add(new Property("HTTP_PORT", "" + httpPort));
-
- cluster.setEnv(new Env(clusterProperties));
- }
-
- private static String injectAsterixPropertyFile(String origZipFile, AsterixInstance asterixInstance)
- throws IOException, JAXBException {
- writeAsterixConfigurationFile(asterixInstance);
- String asterixInstanceDir = InstallerDriver.getAsterixDir() + File.separator + asterixInstance.getName();
- unzip(origZipFile, asterixInstanceDir);
- File sourceJar = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
- + asterixInstance.getAsterixVersion() + ".jar");
- File replacementFile = new File(asterixInstanceDir + File.separator + ASTERIX_CONFIGURATION_FILE);
- replaceInJar(sourceJar, ASTERIX_CONFIGURATION_FILE, replacementFile);
- new File(asterixInstanceDir + File.separator + ASTERIX_CONFIGURATION_FILE).delete();
- String asterixZipName = InstallerDriver.getAsterixZip().substring(
- InstallerDriver.getAsterixZip().lastIndexOf(File.separator) + 1);
- zipDir(new File(asterixInstanceDir), new File(asterixInstanceDir + File.separator + asterixZipName));
- return asterixInstanceDir + File.separator + asterixZipName;
- }
-
- private static String injectAsterixLogPropertyFile(String origZipFile, AsterixInstance asterixInstance)
- throws IOException, InstallerException {
- String asterixInstanceDir = InstallerDriver.getAsterixDir() + File.separator + asterixInstance.getName();
- unzip(origZipFile, asterixInstanceDir);
- File sourceJar1 = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
- + asterixInstance.getAsterixVersion() + ".jar");
- Properties txnLogProperties = new Properties();
- URLClassLoader urlClassLoader = new URLClassLoader(new URL[] { sourceJar1.toURI().toURL() });
- InputStream in = urlClassLoader.getResourceAsStream(TXN_LOG_CONFIGURATION_FILE);
- if (in != null) {
- txnLogProperties.load(in);
- }
-
- writeAsterixLogConfigurationFile(asterixInstance, txnLogProperties);
-
- File sourceJar2 = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
- + asterixInstance.getAsterixVersion() + ".jar");
- File replacementFile = new File(asterixInstanceDir + File.separator + "log.properties");
- replaceInJar(sourceJar2, TXN_LOG_CONFIGURATION_FILE, replacementFile);
-
- new File(asterixInstanceDir + File.separator + "log.properties").delete();
- String asterixZipName = InstallerDriver.getAsterixZip().substring(
- InstallerDriver.getAsterixZip().lastIndexOf(File.separator) + 1);
- zipDir(new File(asterixInstanceDir), new File(asterixInstanceDir + File.separator + asterixZipName));
- return asterixInstanceDir + File.separator + asterixZipName;
- }
-
- public static void addLibraryToAsterixZip(AsterixInstance asterixInstance, String dataverseName,
- String libraryName, String libraryPath) throws IOException {
- File instanceDir = new File(InstallerDriver.getAsterixDir() + File.separator + asterixInstance.getName());
- if (!instanceDir.exists()) {
- instanceDir.mkdirs();
- }
- String asterixZipName = InstallerDriver.getAsterixZip().substring(
- InstallerDriver.getAsterixZip().lastIndexOf(File.separator) + 1);
-
- String sourceZip = instanceDir.getAbsolutePath() + File.separator + asterixZipName;
- unzip(sourceZip, instanceDir.getAbsolutePath());
- File libraryPathInZip = new File(instanceDir.getAbsolutePath() + File.separator + "external" + File.separator
- + "library" + dataverseName + File.separator + "to-add" + File.separator + libraryName);
- libraryPathInZip.mkdirs();
- Runtime.getRuntime().exec("cp" + " " + libraryPath + " " + libraryPathInZip.getAbsolutePath());
- Runtime.getRuntime().exec("rm " + sourceZip);
- String destZip = InstallerDriver.getAsterixDir() + File.separator + asterixInstance.getName() + File.separator
- + asterixZipName;
- zipDir(instanceDir, new File(destZip));
- Runtime.getRuntime().exec("mv" + " " + destZip + " " + sourceZip);
- }
-
- private static Node getMetadataNode(Cluster cluster) {
- Random random = new Random();
- int nNodes = cluster.getNode().size();
- return cluster.getNode().get(random.nextInt(nNodes));
- }
-
public static String getNodeDirectories(String asterixInstanceName, Node node, Cluster cluster) {
String storeDataSubDir = asterixInstanceName + File.separator + "data" + File.separator;
String[] storeDirs = null;
@@ -224,83 +62,10 @@
return nodeDataStore.toString();
}
- private static void writeAsterixConfigurationFile(AsterixInstance asterixInstance) throws IOException,
- JAXBException {
- String asterixInstanceName = asterixInstance.getName();
- Cluster cluster = asterixInstance.getCluster();
- String metadataNodeId = asterixInstance.getMetadataNodeId();
-
- AsterixConfiguration configuration = asterixInstance.getAsterixConfiguration();
- configuration.setMetadataNode(asterixInstanceName + "_" + metadataNodeId);
-
- String storeDir = null;
- List<Store> stores = new ArrayList<Store>();
- for (Node node : cluster.getNode()) {
- storeDir = node.getStore() == null ? cluster.getStore() : node.getStore();
- stores.add(new Store(asterixInstanceName + "_" + node.getId(), storeDir));
- }
- configuration.setStore(stores);
-
- List<Coredump> coredump = new ArrayList<Coredump>();
- String coredumpDir = null;
- List<TransactionLogDir> txnLogDirs = new ArrayList<TransactionLogDir>();
- String txnLogDir = null;
- for (Node node : cluster.getNode()) {
- coredumpDir = node.getLogDir() == null ? cluster.getLogDir() : node.getLogDir();
- coredump.add(new Coredump(asterixInstanceName + "_" + node.getId(), coredumpDir + File.separator
- + asterixInstanceName + "_" + node.getId()));
-
- txnLogDir = node.getTxnLogDir() == null ? cluster.getTxnLogDir() : node.getTxnLogDir();
- txnLogDirs.add(new TransactionLogDir(asterixInstanceName + "_" + node.getId(), txnLogDir));
- }
- configuration.setCoredump(coredump);
- configuration.setTransactionLogDir(txnLogDirs);
-
- File asterixConfDir = new File(InstallerDriver.getAsterixDir() + File.separator + asterixInstanceName);
- asterixConfDir.mkdirs();
-
- JAXBContext ctx = JAXBContext.newInstance(AsterixConfiguration.class);
- Marshaller marshaller = ctx.createMarshaller();
- marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
- marshaller.marshal(configuration, new FileOutputStream(asterixConfDir + File.separator
- + ASTERIX_CONFIGURATION_FILE));
- }
-
- private static void writeAsterixLogConfigurationFile(AsterixInstance asterixInstance, Properties logProperties)
- throws IOException, InstallerException {
- String asterixInstanceName = asterixInstance.getName();
- Cluster cluster = asterixInstance.getCluster();
- StringBuffer conf = new StringBuffer();
- for (Map.Entry<Object, Object> p : logProperties.entrySet()) {
- conf.append(p.getKey() + "=" + p.getValue() + "\n");
- }
-
- for (Node node : cluster.getNode()) {
- String txnLogDir = node.getTxnLogDir() == null ? cluster.getTxnLogDir() : node.getTxnLogDir();
- if (txnLogDir == null) {
- throw new InstallerException("Transaction log directory (txn_log_dir) not configured for node: "
- + node.getId());
- }
- conf.append(asterixInstanceName + "_" + node.getId() + "." + TXN_LOG_DIR_KEY_SUFFIX + "=" + txnLogDir
- + "\n");
- }
- List<edu.uci.ics.asterix.common.configuration.Property> properties = asterixInstance.getAsterixConfiguration()
- .getProperty();
- for (edu.uci.ics.asterix.common.configuration.Property p : properties) {
- if (p.getName().trim().toLowerCase().contains("log")) {
- conf.append(p.getValue() + "=" + p.getValue());
- }
- }
- dumpToFile(InstallerDriver.getAsterixDir() + File.separator + asterixInstanceName + File.separator
- + "log.properties", conf.toString());
-
- }
-
public static AsterixConfiguration getAsterixConfiguration(String asterixConf) throws FileNotFoundException,
IOException, JAXBException {
if (asterixConf == null) {
- asterixConf = InstallerDriver.getManagixHome() + File.separator
- + InstallerDriver.DEFAULT_ASTERIX_CONFIGURATION_PATH;
+ asterixConf = InstallerDriver.getManagixHome() + File.separator + DEFAULT_ASTERIX_CONFIGURATION_PATH;
}
File file = new File(asterixConf);
JAXBContext ctx = JAXBContext.newInstance(AsterixConfiguration.class);
@@ -309,199 +74,4 @@
return asterixConfiguration;
}
- public static void unzip(String sourceFile, String destDir) throws IOException {
- BufferedOutputStream dest = null;
- FileInputStream fis = new FileInputStream(sourceFile);
- ZipInputStream zis = new ZipInputStream(new BufferedInputStream(fis));
- ZipEntry entry = null;
-
- int BUFFER_SIZE = 4096;
- while ((entry = zis.getNextEntry()) != null) {
- String dst = destDir + File.separator + entry.getName();
- if (entry.isDirectory()) {
- createDir(destDir, entry);
- continue;
- }
- int count;
- byte data[] = new byte[BUFFER_SIZE];
-
- // write the file to the disk
- FileOutputStream fos = new FileOutputStream(dst);
- dest = new BufferedOutputStream(fos, BUFFER_SIZE);
- while ((count = zis.read(data, 0, BUFFER_SIZE)) != -1) {
- dest.write(data, 0, count);
- }
- // close the output streams
- dest.flush();
- dest.close();
- }
-
- zis.close();
- }
-
- public static void zipDir(File sourceDir, File destFile) throws IOException {
- FileOutputStream fos = new FileOutputStream(destFile);
- ZipOutputStream zos = new ZipOutputStream(fos);
- zipDir(sourceDir, destFile, zos);
- zos.close();
- }
-
- private static void zipDir(File sourceDir, final File destFile, ZipOutputStream zos) throws IOException {
- File[] dirList = sourceDir.listFiles(new FileFilter() {
- public boolean accept(File f) {
- return !f.getName().endsWith(destFile.getName());
- }
- });
- for (int i = 0; i < dirList.length; i++) {
- File f = dirList[i];
- if (f.isDirectory()) {
- zipDir(f, destFile, zos);
- } else {
- int bytesIn = 0;
- byte[] readBuffer = new byte[2156];
- FileInputStream fis = new FileInputStream(f);
- ZipEntry entry = new ZipEntry(sourceDir.getName() + File.separator + f.getName());
- zos.putNextEntry(entry);
- while ((bytesIn = fis.read(readBuffer)) != -1) {
- zos.write(readBuffer, 0, bytesIn);
- }
- fis.close();
- }
- }
- }
-
- private static void replaceInJar(File sourceJar, String origFile, File replacementFile) throws IOException {
- File destJar = new File(sourceJar.getAbsolutePath() + ".modified");
- InputStream jarIs = null;
- FileInputStream fis = new FileInputStream(replacementFile);
- JarFile sourceJarFile = new JarFile(sourceJar);
- Enumeration<JarEntry> entries = sourceJarFile.entries();
- JarOutputStream jos = new JarOutputStream(new FileOutputStream(destJar));
- byte[] buffer = new byte[2048];
- int read;
- while (entries.hasMoreElements()) {
- JarEntry entry = (JarEntry) entries.nextElement();
- String name = entry.getName();
- if (name.equals(origFile)) {
- continue;
- }
- jarIs = sourceJarFile.getInputStream(entry);
- jos.putNextEntry(entry);
- while ((read = jarIs.read(buffer)) != -1) {
- jos.write(buffer, 0, read);
- }
- }
- JarEntry entry = new JarEntry(origFile);
- jos.putNextEntry(entry);
- while ((read = fis.read(buffer)) != -1) {
- jos.write(buffer, 0, read);
- }
- fis.close();
- jos.close();
- jarIs.close();
- sourceJar.delete();
- destJar.renameTo(sourceJar);
- sourceJar.setExecutable(true);
- }
-
- public static void dumpToFile(String dest, String content) throws IOException {
- FileWriter writer = new FileWriter(dest);
- writer.write(content);
- writer.close();
- }
-
- private static void createDir(String destDirectory, ZipEntry entry) {
- String name = entry.getName();
- int index = name.lastIndexOf(File.separator);
- String dirSequence = name.substring(0, index);
- File newDirs = new File(destDirectory + File.separator + dirSequence);
- newDirs.mkdirs();
- }
-
- public static AsterixInstance validateAsterixInstanceExists(String name, State... permissibleStates)
- throws Exception {
- AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService().getAsterixInstance(name);
- if (instance == null) {
- throw new InstallerException("Asterix instance by name " + name + " does not exist.");
- }
- boolean valid = false;
- for (State state : permissibleStates) {
- if (state.equals(instance.getState())) {
- valid = true;
- break;
- }
- }
- if (!valid) {
- throw new InstallerException("Asterix instance by the name " + name + " is in " + instance.getState()
- + " state ");
- }
- return instance;
- }
-
- public static void validateAsterixInstanceNotExists(String name) throws Exception {
- AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService().getAsterixInstance(name);
- if (instance != null) {
- throw new InstallerException("Asterix instance by name " + name + " already exists.");
- }
- }
-
- public static void evaluateConflictWithOtherInstances(AsterixInstance instance) throws Exception {
- List<AsterixInstance> existingInstances = ServiceProvider.INSTANCE.getLookupService().getAsterixInstances();
- List<String> usedIps = new ArrayList<String>();
- String masterIp = instance.getCluster().getMasterNode().getClusterIp();
- for (Node node : instance.getCluster().getNode()) {
- usedIps.add(node.getClusterIp());
- }
- usedIps.add(instance.getCluster().getMasterNode().getClusterIp());
- boolean conflictFound = false;
- AsterixInstance conflictingInstance = null;
- for (AsterixInstance existing : existingInstances) {
- if (!existing.getState().equals(State.INACTIVE)) {
- conflictFound = existing.getCluster().getMasterNode().getClusterIp().equals(masterIp);
- if (conflictFound) {
- conflictingInstance = existing;
- break;
- }
- for (Node n : existing.getCluster().getNode()) {
- if (usedIps.contains(n.getClusterIp())) {
- conflictFound = true;
- conflictingInstance = existing;
- break;
- }
- }
- }
- }
- if (conflictFound) {
- throw new Exception("Cluster definition conflicts with an existing instance of Asterix: "
- + conflictingInstance.getName());
- }
- }
-
- public static void deleteDirectory(String path) throws IOException {
- Runtime.getRuntime().exec("rm -rf " + path);
- }
-
- public static String executeLocalScript(String path, List<String> args) throws Exception {
- List<String> pargs = new ArrayList<String>();
- pargs.add("/bin/bash");
- pargs.add(path);
- if (args != null) {
- pargs.addAll(args);
- }
- ProcessBuilder pb = new ProcessBuilder(pargs);
- pb.environment().putAll(EventDriver.getEnvironment());
- pb.environment().put("IP_LOCATION", EventDriver.CLIENT_NODE.getClusterIp());
- Process p = pb.start();
- BufferedInputStream bis = new BufferedInputStream(p.getInputStream());
- StringWriter writer = new StringWriter();
- IOUtils.copy(bis, writer, "UTF-8");
- return writer.toString();
- }
-
- public static EventrixClient getEventrixClient(Cluster cluster) throws Exception {
- return new EventrixClient(
- InstallerDriver.getManagixHome() + File.separator + InstallerDriver.MANAGIX_EVENT_DIR, cluster, false,
- OutputHandler.INSTANCE);
- }
-
}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/OutputHandler.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/OutputHandler.java
index 313fa50..0a86196 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/OutputHandler.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/OutputHandler.java
@@ -20,8 +20,8 @@
import edu.uci.ics.asterix.event.management.IOutputHandler;
import edu.uci.ics.asterix.event.management.OutputAnalysis;
+import edu.uci.ics.asterix.event.model.EventList.EventType;
import edu.uci.ics.asterix.event.schema.pattern.Event;
-import edu.uci.ics.asterix.installer.model.EventList.EventType;
public class OutputHandler implements IOutputHandler {
diff --git a/asterix-installer/src/main/resources/hadoop-0.20.2/bin/hadoop b/asterix-installer/src/main/resources/hadoop-0.20.2/bin/hadoop
old mode 100755
new mode 100644
diff --git a/asterix-installer/src/main/resources/hadoop-0.20.2/bin/hadoop-config.sh b/asterix-installer/src/main/resources/hadoop-0.20.2/bin/hadoop-config.sh
old mode 100755
new mode 100644
diff --git a/asterix-installer/src/main/resources/scripts/managix b/asterix-installer/src/main/resources/scripts/managix
old mode 100755
new mode 100644
diff --git a/asterix-installer/src/main/resources/scripts/validate_ssh.sh b/asterix-installer/src/main/resources/scripts/validate_ssh.sh
old mode 100755
new mode 100644
diff --git a/asterix-installer/src/main/resources/scripts/verify.sh b/asterix-installer/src/main/resources/scripts/verify.sh
old mode 100755
new mode 100644
diff --git a/asterix-installer/src/main/resources/zookeeper/start_zk.sh b/asterix-installer/src/main/resources/zookeeper/start_zk.sh
old mode 100755
new mode 100644
diff --git a/asterix-installer/src/main/resources/zookeeper/stop_zk b/asterix-installer/src/main/resources/zookeeper/stop_zk
old mode 100755
new mode 100644
diff --git a/asterix-installer/src/main/resources/zookeeper/zk.init b/asterix-installer/src/main/resources/zookeeper/zk.init
old mode 100755
new mode 100644
diff --git a/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixExternalLibraryIT.java b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixExternalLibraryIT.java
new file mode 100644
index 0000000..daa8bb5
--- /dev/null
+++ b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixExternalLibraryIT.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.installer.test;
+
+import java.io.File;
+import java.util.List;
+import java.util.logging.Logger;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.test.aql.TestsUtils;
+import edu.uci.ics.asterix.testframework.context.TestCaseContext;
+
+public class AsterixExternalLibraryIT {
+
+ private static final String LIBRARY_NAME = "testlib";
+ private static final String LIBRARY_DATAVERSE = "externallibtest";
+ private static final String PATH_BASE = "src/test/resources/integrationts/library";
+ private static final String PATH_ACTUAL = "ittest/";
+ private static final String LIBRARY_PATH = "asterix-external-data"
+ + File.separator + "target" + File.separator
+ + "testlib-zip-binary-assembly.zip";
+ private static final Logger LOGGER = Logger
+ .getLogger(AsterixExternalLibraryIT.class.getName());
+ private static List<TestCaseContext> testCaseCollection;
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ AsterixInstallerIntegrationUtil.init();
+ File asterixInstallerProjectDir = new File(System
+ .getProperty("user.dir"));
+ String asterixExternalLibraryPath = asterixInstallerProjectDir
+ .getParentFile().getAbsolutePath()
+ + File.separator + LIBRARY_PATH;
+ LOGGER.info("Installing library :" + LIBRARY_NAME + " located at "
+ + asterixExternalLibraryPath + " in dataverse "
+ + LIBRARY_DATAVERSE);
+ AsterixInstallerIntegrationUtil.installLibrary(LIBRARY_NAME,
+ LIBRARY_DATAVERSE, asterixExternalLibraryPath);
+ AsterixInstallerIntegrationUtil
+ .transformIntoRequiredState(State.ACTIVE);
+ TestCaseContext.Builder b = new TestCaseContext.Builder();
+ testCaseCollection = b.build(new File(PATH_BASE));
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ AsterixInstallerIntegrationUtil.deinit();
+ }
+
+ @Test
+ public void test() throws Exception {
+ for (TestCaseContext testCaseCtx : testCaseCollection) {
+ TestsUtils.executeTest(PATH_ACTUAL, testCaseCtx);
+ }
+ }
+
+ public static void main(String[] args) throws Exception {
+ try {
+ setUp();
+ new AsterixExternalLibraryIT().test();
+ } catch (Exception e) {
+ e.printStackTrace();
+ LOGGER.info("TEST CASES FAILED");
+ } finally {
+ tearDown();
+ }
+ }
+
+}
diff --git a/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixFaultToleranceIT.java b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixFaultToleranceIT.java
new file mode 100644
index 0000000..8a126a2
--- /dev/null
+++ b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixFaultToleranceIT.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.installer.test;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.logging.Logger;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runners.Parameterized.Parameters;
+
+import edu.uci.ics.asterix.test.aql.TestsUtils;
+import edu.uci.ics.asterix.testframework.context.TestCaseContext;
+
+public class AsterixFaultToleranceIT {
+
+ private static final String PATH_BASE = "src/test/resources/integrationts/fault-tolerance";
+ private static final String PATH_ACTUAL = "ittest/";
+ private static final Logger LOGGER = Logger.getLogger(AsterixFaultToleranceIT.class.getName());
+ private static List<TestCaseContext> testCaseCollection;
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ AsterixInstallerIntegrationUtil.init();
+ TestCaseContext.Builder b = new TestCaseContext.Builder();
+ testCaseCollection = b.build(new File(PATH_BASE));
+ File outdir = new File(PATH_ACTUAL);
+ outdir.mkdirs();
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ AsterixInstallerIntegrationUtil.deinit();
+ File outdir = new File(PATH_ACTUAL);
+ File[] files = outdir.listFiles();
+ if (files == null || files.length == 0) {
+ outdir.delete();
+ }
+ }
+
+ @Parameters
+ public static Collection<Object[]> tests() throws Exception {
+ Collection<Object[]> testArgs = new ArrayList<Object[]>();
+ return testArgs;
+ }
+
+ @Test
+ public void test() throws Exception {
+ for (TestCaseContext testCaseCtx : testCaseCollection) {
+ TestsUtils.executeTest(PATH_ACTUAL, testCaseCtx);
+ }
+ }
+
+ public static void main(String[] args) throws Exception {
+ try {
+ setUp();
+ new AsterixFaultToleranceIT().test();
+ } catch (Exception e) {
+ e.printStackTrace();
+ LOGGER.info("TEST CASE(S) FAILED");
+ } finally {
+ tearDown();
+ }
+ }
+
+}
diff --git a/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixInstallerIntegrationUtil.java b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixInstallerIntegrationUtil.java
index c7beb28..1096a4d 100644
--- a/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixInstallerIntegrationUtil.java
+++ b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixInstallerIntegrationUtil.java
@@ -28,191 +28,210 @@
import javax.xml.bind.Marshaller;
import javax.xml.bind.Unmarshaller;
+import edu.uci.ics.asterix.event.error.VerificationUtil;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.model.AsterixRuntimeState;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
import edu.uci.ics.asterix.installer.command.CommandHandler;
-import edu.uci.ics.asterix.installer.command.ShutdownCommand;
import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.error.VerificationUtil;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.model.AsterixRuntimeState;
import edu.uci.ics.asterix.installer.schema.conf.Configuration;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
public class AsterixInstallerIntegrationUtil {
- private static String managixHome;
- private static String clusterConfigurationPath;
- private static final CommandHandler cmdHandler = new CommandHandler();
- public static final String ASTERIX_INSTANCE_NAME = "asterix";
- private static final String CC_IP_ADDRESS = "127.0.0.1";
- private static final int DEFAULT_HYRACKS_CC_CLIENT_PORT = 1098;
- private static final int zookeeperClientPort = 2900;
- private static final int zookeeperTestClientPort = 3945;
+ private static String managixHome;
+ private static String clusterConfigurationPath;
+ private static final CommandHandler cmdHandler = new CommandHandler();
+ public static final String ASTERIX_INSTANCE_NAME = "asterix";
+ private static final String CC_IP_ADDRESS = "127.0.0.1";
+ private static final int DEFAULT_HYRACKS_CC_CLIENT_PORT = 1098;
+ private static final int zookeeperClientPort = 2900;
+ private static final int zookeeperTestClientPort = 3945;
- private static IHyracksClientConnection hcc;
+ private static IHyracksClientConnection hcc;
- private static final Logger LOGGER = Logger.getLogger(AsterixInstallerIntegrationUtil.class.getName());
+ private static final Logger LOGGER = Logger
+ .getLogger(AsterixInstallerIntegrationUtil.class.getName());
- public static void deinit() throws Exception {
- deleteInstance();
- stopZookeeper();
- }
+ public static void deinit() throws Exception {
+ deleteInstance();
+ stopZookeeper();
+ }
- public static void init() throws Exception {
- File asterixProjectDir = new File(System.getProperty("user.dir"));
- File installerTargetDir = new File(asterixProjectDir, "target");
- String managixHomeDirName = installerTargetDir.list(new FilenameFilter() {
- @Override
- public boolean accept(File dir, String name) {
- return new File(dir, name).isDirectory() && name.startsWith("asterix-installer")
- && name.endsWith("binary-assembly");
- }
+ public static void init() throws Exception {
+ File asterixProjectDir = new File(System.getProperty("user.dir"));
+ File installerTargetDir = new File(asterixProjectDir, "target");
+ String managixHomeDirName = installerTargetDir
+ .list(new FilenameFilter() {
+ @Override
+ public boolean accept(File dir, String name) {
+ return new File(dir, name).isDirectory()
+ && name.startsWith("asterix-installer")
+ && name.endsWith("binary-assembly");
+ }
- })[0];
- managixHome = new File(installerTargetDir, managixHomeDirName).getAbsolutePath();
- System.setProperty("log4j.configuration", managixHome + File.separator + "conf" + File.separator
- + "log4j.properties");
+ })[0];
+ managixHome = new File(installerTargetDir, managixHomeDirName)
+ .getAbsolutePath();
+ System.setProperty("log4j.configuration", managixHome + File.separator
+ + "conf" + File.separator + "log4j.properties");
- managixHome = AsterixInstallerIntegrationUtil.getManagixHome();
- clusterConfigurationPath = managixHome + File.separator + "clusters" + File.separator + "local"
- + File.separator + "local.xml";
+ managixHome = AsterixInstallerIntegrationUtil.getManagixHome();
+ clusterConfigurationPath = managixHome + File.separator + "clusters"
+ + File.separator + "local" + File.separator + "local.xml";
- InstallerDriver.setManagixHome(managixHome);
+ InstallerDriver.setManagixHome(managixHome);
- String command = "configure";
- cmdHandler.processCommand(command.split(" "));
- command = "validate -c " + clusterConfigurationPath;
- cmdHandler.processCommand(command.split(" "));
+ String command = "configure";
+ cmdHandler.processCommand(command.split(" "));
+ command = "validate -c " + clusterConfigurationPath;
+ cmdHandler.processCommand(command.split(" "));
- startZookeeper();
- InstallerDriver.initConfig(true);
- createInstance();
- hcc = new HyracksConnection(CC_IP_ADDRESS, DEFAULT_HYRACKS_CC_CLIENT_PORT);
- }
+ startZookeeper();
+ Thread.sleep(2000);
+ InstallerDriver.initConfig(true);
+ createInstance();
+ hcc = new HyracksConnection(CC_IP_ADDRESS,
+ DEFAULT_HYRACKS_CC_CLIENT_PORT);
+ }
- public static IHyracksClientConnection getHyracksConnection() {
- return hcc;
- }
+ public static IHyracksClientConnection getHyracksConnection() {
+ return hcc;
+ }
- private static void startZookeeper() throws Exception {
- initZookeeperTestConfiguration(zookeeperClientPort);
- String script = managixHome + File.separator + "bin" + File.separator + "managix";
+ private static void startZookeeper() throws Exception {
+ initZookeeperTestConfiguration(zookeeperClientPort);
+ String script = managixHome + File.separator + "bin" + File.separator
+ + "managix";
- // shutdown zookeeper if running
- String command = "shutdown";
- cmdHandler.processCommand(command.split(" "));
+ // shutdown zookeeper if running
+ String command = "shutdown";
+ cmdHandler.processCommand(command.split(" "));
- Thread.sleep(2000);
+ Thread.sleep(2000);
- // start zookeeper
- initZookeeperTestConfiguration(zookeeperTestClientPort);
- ProcessBuilder pb2 = new ProcessBuilder(script, "describe");
- Map<String, String> env2 = pb2.environment();
- env2.put("MANAGIX_HOME", managixHome);
- pb2.start();
+ // start zookeeper
+ initZookeeperTestConfiguration(zookeeperTestClientPort);
+ ProcessBuilder pb2 = new ProcessBuilder(script, "describe");
+ Map<String, String> env2 = pb2.environment();
+ env2.put("MANAGIX_HOME", managixHome);
+ pb2.start();
- Thread.sleep(2000);
- }
+ }
- public static void createInstance() throws Exception {
+ public static void createInstance() throws Exception {
- String command = null;
- AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService()
- .getAsterixInstance(ASTERIX_INSTANCE_NAME);
- if (instance != null) {
- transformIntoRequiredState(State.INACTIVE);
- command = "delete -n " + ASTERIX_INSTANCE_NAME;
- cmdHandler.processCommand(command.split(" "));
- }
+ String command = null;
+ AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService()
+ .getAsterixInstance(ASTERIX_INSTANCE_NAME);
+ if (instance != null) {
+ transformIntoRequiredState(State.INACTIVE);
+ command = "delete -n " + ASTERIX_INSTANCE_NAME;
+ cmdHandler.processCommand(command.split(" "));
+ }
- command = "create -n " + ASTERIX_INSTANCE_NAME + " " + "-c" + " " + clusterConfigurationPath;
- cmdHandler.processCommand(command.split(" "));
+ command = "create -n " + ASTERIX_INSTANCE_NAME + " " + "-c" + " "
+ + clusterConfigurationPath;
+ cmdHandler.processCommand(command.split(" "));
- instance = ServiceProvider.INSTANCE.getLookupService().getAsterixInstance(ASTERIX_INSTANCE_NAME);
- AsterixRuntimeState state = VerificationUtil.getAsterixRuntimeState(instance);
- assert (state.getFailedNCs().isEmpty() && state.isCcRunning());
- }
+ instance = ServiceProvider.INSTANCE.getLookupService()
+ .getAsterixInstance(ASTERIX_INSTANCE_NAME);
+ AsterixRuntimeState state = VerificationUtil
+ .getAsterixRuntimeState(instance);
+ assert (state.getFailedNCs().isEmpty() && state.isCcRunning());
+ }
- private static void initZookeeperTestConfiguration(int port) throws JAXBException, FileNotFoundException {
- String installerConfPath = InstallerDriver.getManagixHome() + File.separator + InstallerDriver.MANAGIX_CONF_XML;
- JAXBContext ctx = JAXBContext.newInstance(Configuration.class);
- Unmarshaller unmarshaller = ctx.createUnmarshaller();
- Configuration configuration = (Configuration) unmarshaller.unmarshal(new File(installerConfPath));
- configuration.getZookeeper().setClientPort(new BigInteger("" + port));
- Marshaller marshaller = ctx.createMarshaller();
- marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
- marshaller.marshal(configuration, new FileOutputStream(installerConfPath));
- }
+ private static void initZookeeperTestConfiguration(int port)
+ throws JAXBException, FileNotFoundException {
+ String installerConfPath = InstallerDriver.getManagixHome()
+ + File.separator + InstallerDriver.MANAGIX_CONF_XML;
+ JAXBContext ctx = JAXBContext.newInstance(Configuration.class);
+ Unmarshaller unmarshaller = ctx.createUnmarshaller();
+ Configuration configuration = (Configuration) unmarshaller
+ .unmarshal(new File(installerConfPath));
+ configuration.getZookeeper().setClientPort(new BigInteger("" + port));
+ Marshaller marshaller = ctx.createMarshaller();
+ marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
+ marshaller.marshal(configuration, new FileOutputStream(
+ installerConfPath));
+ }
- public static void transformIntoRequiredState(AsterixInstance.State state) throws Exception {
- AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService()
- .getAsterixInstance(ASTERIX_INSTANCE_NAME);
- assert (instance != null);
- if (instance.getState().equals(state)) {
- return;
- }
- if (state.equals(AsterixInstance.State.UNUSABLE)) {
- throw new IllegalArgumentException("Invalid desired state");
- }
+ public static void transformIntoRequiredState(AsterixInstance.State state)
+ throws Exception {
+ AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService()
+ .getAsterixInstance(ASTERIX_INSTANCE_NAME);
+ assert (instance != null);
+ if (instance.getState().equals(state)) {
+ return;
+ }
+ if (state.equals(AsterixInstance.State.UNUSABLE)) {
+ throw new IllegalArgumentException("Invalid desired state");
+ }
- String command = null;
- switch (instance.getState()) {
- case ACTIVE:
- command = "stop -n " + ASTERIX_INSTANCE_NAME;
- break;
- case INACTIVE:
- command = "start -n " + ASTERIX_INSTANCE_NAME;
- break;
- }
- cmdHandler.processCommand(command.split(" "));
- }
+ String command = null;
+ switch (instance.getState()) {
+ case ACTIVE:
+ command = "stop -n " + ASTERIX_INSTANCE_NAME;
+ break;
+ case INACTIVE:
+ command = "start -n " + ASTERIX_INSTANCE_NAME;
+ break;
+ }
+ cmdHandler.processCommand(command.split(" "));
+ }
- private static void stopZookeeper() throws IOException, JAXBException {
- String script = managixHome + File.separator + "bin" + File.separator + "managix";
- // shutdown zookeeper if running
- ProcessBuilder pb = new ProcessBuilder(script, "shutdown");
- Map<String, String> env = pb.environment();
- env.put("MANAGIX_HOME", managixHome);
- pb.start();
- }
+ private static void stopZookeeper() throws IOException, JAXBException {
+ String script = managixHome + File.separator + "bin" + File.separator
+ + "managix";
+ // shutdown zookeeper if running
+ ProcessBuilder pb = new ProcessBuilder(script, "shutdown");
+ Map<String, String> env = pb.environment();
+ env.put("MANAGIX_HOME", managixHome);
+ pb.start();
+ }
- private static void deleteInstance() throws Exception {
- String command = null;
- AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService()
- .getAsterixInstance(ASTERIX_INSTANCE_NAME);
+ private static void deleteInstance() throws Exception {
+ String command = null;
+ AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService()
+ .getAsterixInstance(ASTERIX_INSTANCE_NAME);
- if (instance == null) {
- return;
- } else {
- transformIntoRequiredState(State.INACTIVE);
- command = "delete -n " + ASTERIX_INSTANCE_NAME;
- cmdHandler.processCommand(command.split(" "));
- }
- instance = ServiceProvider.INSTANCE.getLookupService().getAsterixInstance(ASTERIX_INSTANCE_NAME);
- assert (instance == null);
- }
+ if (instance == null) {
+ return;
+ } else {
+ transformIntoRequiredState(State.INACTIVE);
+ command = "delete -n " + ASTERIX_INSTANCE_NAME;
+ cmdHandler.processCommand(command.split(" "));
+ }
+ instance = ServiceProvider.INSTANCE.getLookupService()
+ .getAsterixInstance(ASTERIX_INSTANCE_NAME);
+ assert (instance == null);
+ }
- public static String getManagixHome() {
- return managixHome;
- }
+ public static String getManagixHome() {
+ return managixHome;
+ }
- public static void installLibrary(String libraryName, String libraryDataverse, String libraryPath) throws Exception {
- transformIntoRequiredState(State.INACTIVE);
- String command = "install -n " + ASTERIX_INSTANCE_NAME + " -d " + libraryDataverse + " -l " + libraryName
- + " -p " + libraryPath;
- cmdHandler.processCommand(command.split(" "));
- }
+ public static void installLibrary(String libraryName,
+ String libraryDataverse, String libraryPath) throws Exception {
+ transformIntoRequiredState(State.INACTIVE);
+ String command = "install -n " + ASTERIX_INSTANCE_NAME + " -d "
+ + libraryDataverse + " -l " + libraryName + " -p "
+ + libraryPath;
+ cmdHandler.processCommand(command.split(" "));
+ }
- public static void uninstallLibrary(String dataverseName, String libraryName) throws Exception {
- transformIntoRequiredState(State.INACTIVE);
- String command = "uninstall -n " + ASTERIX_INSTANCE_NAME + " -d " + dataverseName + " -l " + "libraryName";
- cmdHandler.processCommand(command.split(" "));
- }
+ public static void uninstallLibrary(String dataverseName, String libraryName)
+ throws Exception {
+ transformIntoRequiredState(State.INACTIVE);
+ String command = "uninstall -n " + ASTERIX_INSTANCE_NAME + " -d "
+ + dataverseName + " -l " + "libraryName";
+ cmdHandler.processCommand(command.split(" "));
+ }
- public static void executeCommand(String command) throws Exception {
- cmdHandler.processCommand(command.trim().split(" "));
- }
+ public static void executeCommand(String command) throws Exception {
+ cmdHandler.processCommand(command.trim().split(" "));
+ }
}
diff --git a/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixLifecycleIT.java b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixLifecycleIT.java
index c8f6cee..421eb42 100644
--- a/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixLifecycleIT.java
+++ b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixLifecycleIT.java
@@ -25,12 +25,12 @@
import org.junit.Test;
import org.junit.runners.Parameterized.Parameters;
+import edu.uci.ics.asterix.event.error.VerificationUtil;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.model.AsterixInstance.State;
+import edu.uci.ics.asterix.event.model.AsterixRuntimeState;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
import edu.uci.ics.asterix.installer.command.CommandHandler;
-import edu.uci.ics.asterix.installer.error.VerificationUtil;
-import edu.uci.ics.asterix.installer.model.AsterixInstance;
-import edu.uci.ics.asterix.installer.model.AsterixInstance.State;
-import edu.uci.ics.asterix.installer.model.AsterixRuntimeState;
-import edu.uci.ics.asterix.installer.service.ServiceProvider;
import edu.uci.ics.asterix.test.aql.TestsUtils;
import edu.uci.ics.asterix.testframework.context.TestCaseContext;
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.1.ddl.aql b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.1.ddl.aql
new file mode 100644
index 0000000..2b3ce0d
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.1.ddl.aql
@@ -0,0 +1,28 @@
+drop dataverse feeds if exists;
+create dataverse feeds;
+use dataverse feeds;
+
+create type TwitterUserType as closed {
+ screen-name: string,
+ lang: string,
+ friends_count: int32,
+ statuses_count: int32,
+ name: string,
+ followers_count: int32
+}
+
+create type TweetMessageType as closed {
+ tweetid: string,
+ user: TwitterUserType,
+ sender-location: point,
+ send-time: datetime,
+ referred-topics: {{ string }},
+ message-text: string
+}
+
+create dataset Tweets(TweetMessageType)
+primary key tweetid;
+
+create feed TwitterFirehose
+using twitter_firehose
+(("duration"="5"),("tps"="50"));
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.2.update.aql b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.2.update.aql
new file mode 100644
index 0000000..64dbf25
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.2.update.aql
@@ -0,0 +1,3 @@
+use dataverse feeds;
+
+connect feed TwitterFirehose to dataset Tweets;
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.3.mgx.aql b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.3.mgx.aql
new file mode 100644
index 0000000..2d8a23e
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.3.mgx.aql
@@ -0,0 +1 @@
+stop -n asterix
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.4.mgx.aql b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.4.mgx.aql
new file mode 100644
index 0000000..4e99f33
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.4.mgx.aql
@@ -0,0 +1 @@
+start -n asterix
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.5.sleep.aql b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.5.sleep.aql
new file mode 100644
index 0000000..c5da56a
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.5.sleep.aql
@@ -0,0 +1 @@
+40000
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.6.query.aql b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.6.query.aql
new file mode 100644
index 0000000..d03b9fe
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.6.query.aql
@@ -0,0 +1,10 @@
+use dataverse feeds;
+
+let $numTuples:=count(for $x in dataset Tweets
+return $x)
+let $result:=if($numTuples > 225)
+then
+ 1
+else
+ 0
+return $result
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/results/feeds/IN1-cluster-restart/IN1-cluster-restart.1.adm b/asterix-installer/src/test/resources/integrationts/fault-tolerance/results/feeds/IN1-cluster-restart/IN1-cluster-restart.1.adm
new file mode 100644
index 0000000..d00491f
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/results/feeds/IN1-cluster-restart/IN1-cluster-restart.1.adm
@@ -0,0 +1 @@
+1
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/results/feeds/IN1-cluster-restart/IN1-cluster-restart.2.adm b/asterix-installer/src/test/resources/integrationts/fault-tolerance/results/feeds/IN1-cluster-restart/IN1-cluster-restart.2.adm
new file mode 100644
index 0000000..d00491f
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/results/feeds/IN1-cluster-restart/IN1-cluster-restart.2.adm
@@ -0,0 +1 @@
+1
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/testsuite.xml b/asterix-installer/src/test/resources/integrationts/fault-tolerance/testsuite.xml
new file mode 100644
index 0000000..0d9ed23
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/fault-tolerance/testsuite.xml
@@ -0,0 +1,10 @@
+<test-suite xmlns="urn:xml.testframework.asterix.ics.uci.edu" ResultOffsetPath="results" QueryOffsetPath="queries" QueryFileExtension=".aql">
+ <test-group name="fault-tolerance">
+ <test-case FilePath="feeds">
+ <compilation-unit name="IN1-cluster-restart">
+ <output-dir compare="Text">IN1-cluster-restart</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+</test-suite>
+
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/ingest_feed/feed_ingest.1.ddl.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/ingest_feed/feed_ingest.1.ddl.aql
new file mode 100644
index 0000000..edd08ce
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/ingest_feed/feed_ingest.1.ddl.aql
@@ -0,0 +1,25 @@
+/*
+ * Description : Create a feed dataset that uses the feed simulator adapter.
+ The feed simulator simulates feed from a file in the local fs.
+ Associate with the feed an external user-defined function. The UDF
+ finds topics in each tweet. A topic is identified by a #.
+ Begin ingestion and apply external user defined function
+ * Expected Res : Success
+ * Date : 23rd Apr 2013
+ */
+use dataverse externallibtest;
+
+create type TweetType as closed {
+ id: string,
+ username : string,
+ location : string,
+ text : string,
+ timestamp : string,
+ topics : {{string}}?
+}
+
+create feed dataset TweetFeed(TweetType)
+using "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory"
+(("output-type-name"="TweetType"),("fs"="localfs"),("path"="nc1://../asterix-app/data/twitter/obamatweets.adm"),("format"="adm"),("tuple-interval"="10"))
+apply function testlib#parseTweet@1
+primary key id;
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/ingest_feed/feed_ingest.2.update.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/ingest_feed/feed_ingest.2.update.aql
new file mode 100644
index 0000000..b7e03ec
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/ingest_feed/feed_ingest.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description : Create a feed dataset that uses the feed simulator adapter.
+ The feed simulator simulates feed from a file in the local fs.
+ Associate with the feed an external user-defined function. The UDF
+ finds topics in each tweet. A topic is identified by a #.
+ Begin ingestion and apply external user defined function
+ * Expected Res : Success
+ * Date : 23rd Apr 2013
+ */
+use dataverse externallibtest;
+
+begin feed TweetFeed;
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/ingest_feed/feed_ingest.3.query.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/ingest_feed/feed_ingest.3.query.aql
new file mode 100644
index 0000000..fd36fbc
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-feeds/ingest_feed/feed_ingest.3.query.aql
@@ -0,0 +1,13 @@
+/*
+ * Description : Create a feed dataset that uses the feed simulator adapter.
+ The feed simulator simulates feed from a file in the local fs.
+ Associate with the feed an external user-defined function. The UDF
+ finds topics in each tweet. A topic is identified by a #.
+ Begin ingestion and apply external user defined function
+ * Expected Res : Success
+ * Date : 23rd Apr 2013
+ */
+use dataverse externallibtest;
+
+for $x in dataset TweetFeed
+return $x
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/getCapital/getCapital.1.ddl.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/getCapital/getCapital.1.ddl.aql
new file mode 100644
index 0000000..e140d9a
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/getCapital/getCapital.1.ddl.aql
@@ -0,0 +1,6 @@
+use dataverse externallibtest;
+
+create type CountryCapitalType if not exists as closed {
+country: string,
+capital: string
+};
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/getCapital/getCapital.2.query.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/getCapital/getCapital.2.query.aql
new file mode 100644
index 0000000..256a3cd
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/getCapital/getCapital.2.query.aql
@@ -0,0 +1,5 @@
+use dataverse externallibtest;
+
+let $input:=["England","Italy","China","United States","India","Jupiter"]
+for $country in $input
+return testlib#getCapital($country)
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.1.ddl.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.1.ddl.aql
new file mode 100644
index 0000000..11a5ddc
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.1.ddl.aql
@@ -0,0 +1,9 @@
+use dataverse externallibtest;
+
+create type TextType if not exists as closed {
+id: int32,
+text: string
+};
+
+create dataset Check(TextType)
+primary key id;
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.2.update.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.2.update.aql
new file mode 100644
index 0000000..8a14669
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.2.update.aql
@@ -0,0 +1,6 @@
+use dataverse externallibtest;
+
+insert into dataset Check (
+{"id": 1, "text":"university of california, irvine"}
+);
+
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.3.update.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.3.update.aql
new file mode 100644
index 0000000..36f3133
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.3.update.aql
@@ -0,0 +1,7 @@
+use dataverse externallibtest;
+
+insert into dataset Check (
+ for $x in dataset Check
+ let $y:=testlib#toUpper($x)
+ return $y
+);
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.4.query.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.4.query.aql
new file mode 100644
index 0000000..997c333
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/insert-from-select/insert-from-select.4.query.aql
@@ -0,0 +1,6 @@
+use dataverse externallibtest;
+
+for $x in dataset Check
+where $x.id < 0
+order by $x.id
+return $x
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/mysum/mysum.1.query.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/mysum/mysum.1.query.aql
new file mode 100644
index 0000000..ce255b8
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/mysum/mysum.1.query.aql
@@ -0,0 +1,4 @@
+use dataverse externallibtest;
+
+let $x:=testlib#mysum(3,4)
+return $x
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/toUpper/toUpper.1.ddl.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/toUpper/toUpper.1.ddl.aql
new file mode 100644
index 0000000..67635f5
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/toUpper/toUpper.1.ddl.aql
@@ -0,0 +1,7 @@
+use dataverse externallibtest;
+
+create type TextType if not exists as closed {
+id: int32,
+text: string
+};
+
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/toUpper/toUpper.2.query.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/toUpper/toUpper.2.query.aql
new file mode 100644
index 0000000..a742203
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-functions/toUpper/toUpper.2.query.aql
@@ -0,0 +1,5 @@
+use dataverse externallibtest;
+
+let $input:={"id": 1, "text":"university of california, irvine"}
+let $x:=testlib#toUpper($input)
+return $x
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-metadata/dataverseDataset/dataverseDataset.1.query.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-metadata/dataverseDataset/dataverseDataset.1.query.aql
new file mode 100644
index 0000000..40316d8
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-metadata/dataverseDataset/dataverseDataset.1.query.aql
@@ -0,0 +1,3 @@
+for $x in dataset Metadata.Dataverse
+order by $x.DataverseName
+return $x
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-metadata/functionDataset/functionDataset.1.query.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-metadata/functionDataset/functionDataset.1.query.aql
new file mode 100644
index 0000000..fc47972
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-metadata/functionDataset/functionDataset.1.query.aql
@@ -0,0 +1,3 @@
+for $x in dataset Metadata.Function
+order by $x.Name
+return $x
diff --git a/asterix-installer/src/test/resources/integrationts/library/queries/library-metadata/libraryDataset/libraryDataset.1.query.aql b/asterix-installer/src/test/resources/integrationts/library/queries/library-metadata/libraryDataset/libraryDataset.1.query.aql
new file mode 100644
index 0000000..36a8a52
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/queries/library-metadata/libraryDataset/libraryDataset.1.query.aql
@@ -0,0 +1,3 @@
+for $x in dataset Metadata.Library
+order by $x.Name
+return $x
diff --git a/asterix-installer/src/test/resources/integrationts/library/results/library-functions/getCapital/getCapital.1.adm b/asterix-installer/src/test/resources/integrationts/library/results/library-functions/getCapital/getCapital.1.adm
new file mode 100644
index 0000000..16e9591
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/results/library-functions/getCapital/getCapital.1.adm
@@ -0,0 +1,6 @@
+{ "country": "England", "capital": "London" }
+{ "country": "Italy", "capital": "Rome" }
+{ "country": "China", "capital": "Beijing" }
+{ "country": "United States", "capital": "Washington D.C." }
+{ "country": "India", "capital": "New Delhi" }
+{ "country": "Jupiter", "capital": "NOT_FOUND" }
diff --git a/asterix-installer/src/test/resources/integrationts/library/results/library-functions/insert-from-select/insert-from-select.1.adm b/asterix-installer/src/test/resources/integrationts/library/results/library-functions/insert-from-select/insert-from-select.1.adm
new file mode 100644
index 0000000..a839cbc
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/results/library-functions/insert-from-select/insert-from-select.1.adm
@@ -0,0 +1 @@
+{ "id": -1, "text": "UNIVERSITY OF CALIFORNIA, IRVINE" }
diff --git a/asterix-installer/src/test/resources/integrationts/library/results/library-functions/mysum/mysum.1.adm b/asterix-installer/src/test/resources/integrationts/library/results/library-functions/mysum/mysum.1.adm
new file mode 100644
index 0000000..7f8f011
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/results/library-functions/mysum/mysum.1.adm
@@ -0,0 +1 @@
+7
diff --git a/asterix-installer/src/test/resources/integrationts/library/results/library-functions/toUpper/toUpper.1.adm b/asterix-installer/src/test/resources/integrationts/library/results/library-functions/toUpper/toUpper.1.adm
new file mode 100644
index 0000000..a839cbc
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/results/library-functions/toUpper/toUpper.1.adm
@@ -0,0 +1 @@
+{ "id": -1, "text": "UNIVERSITY OF CALIFORNIA, IRVINE" }
diff --git a/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/dataverseDataset/dataverseDataset.1.adm b/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/dataverseDataset/dataverseDataset.1.adm
new file mode 100644
index 0000000..330c347
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/dataverseDataset/dataverseDataset.1.adm
@@ -0,0 +1,2 @@
+{ "DataverseName": "Metadata", "DataFormat": "edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Thu Apr 25 11:17:56 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "externallibtest", "DataFormat": "edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Thu Apr 25 11:18:12 PDT 2013", "PendingOp": 0 }
diff --git a/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/functionDataset/functionDataset.1.adm b/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/functionDataset/functionDataset.1.adm
new file mode 100644
index 0000000..bb42e62
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/functionDataset/functionDataset.1.adm
@@ -0,0 +1,12 @@
+{ "DataverseName": "externallibtest", "Name": "testlib#allTypes", "Arity": "1", "Params": [ "AllType" ], "ReturnType": "AllType", "Definition": "edu.uci.ics.asterix.external.library.AllTypesFactory
+ ", "Language": "JAVA", "Kind": "SCALAR" }
+{ "DataverseName": "externallibtest", "Name": "testlib#echoDelay", "Arity": "1", "Params": [ "TweetMessageType" ], "ReturnType": "TweetMessageType", "Definition": "edu.uci.ics.asterix.external.library.EchoDelayFactory
+ ", "Language": "JAVA", "Kind": "SCALAR" }
+{ "DataverseName": "externallibtest", "Name": "testlib#getCapital", "Arity": "1", "Params": [ "ASTRING" ], "ReturnType": "CountryCapitalType", "Definition": "edu.uci.ics.asterix.external.library.CapitalFinderFactory
+ ", "Language": "JAVA", "Kind": "SCALAR" }
+{ "DataverseName": "externallibtest", "Name": "testlib#mysum", "Arity": "2", "Params": [ "AINT32", "AINT32" ], "ReturnType": "AINT32", "Definition": "edu.uci.ics.asterix.external.library.SumFactory
+ ", "Language": "JAVA", "Kind": "SCALAR" }
+{ "DataverseName": "externallibtest", "Name": "testlib#parseTweet", "Arity": "1", "Params": [ "TweetType" ], "ReturnType": "TweetType", "Definition": "edu.uci.ics.asterix.external.library.ParseTweetFactory
+ ", "Language": "JAVA", "Kind": "SCALAR" }
+{ "DataverseName": "externallibtest", "Name": "testlib#toUpper", "Arity": "1", "Params": [ "TextType" ], "ReturnType": "TextType", "Definition": "edu.uci.ics.asterix.external.library.UpperCaseFactory
+ ", "Language": "JAVA", "Kind": "SCALAR" }
diff --git a/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/libraryDataset/libraryDataset.1.adm b/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/libraryDataset/libraryDataset.1.adm
new file mode 100644
index 0000000..573db0c
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/libraryDataset/libraryDataset.1.adm
@@ -0,0 +1 @@
+{ "DataverseName": "externallibtest", "Name": "testlib", "Timestamp": "Mon Apr 22 23:36:55 PDT 2013" }
diff --git a/asterix-installer/src/test/resources/integrationts/library/testsuite.xml b/asterix-installer/src/test/resources/integrationts/library/testsuite.xml
new file mode 100644
index 0000000..f97c050
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/library/testsuite.xml
@@ -0,0 +1,42 @@
+<test-suite xmlns="urn:xml.testframework.asterix.ics.uci.edu" ResultOffsetPath="results" QueryOffsetPath="queries" QueryFileExtension=".aql">
+ <test-group name="library-functions">
+ <test-case FilePath="library-functions">
+ <compilation-unit name="mysum">
+ <output-dir compare="Text">mysum</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="library-functions">
+ <compilation-unit name="toUpper">
+ <output-dir compare="Text">toUpper</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="library-functions">
+ <compilation-unit name="insert-from-select">
+ <output-dir compare="Text">insert-from-select</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="library-functions">
+ <compilation-unit name="getCapital">
+ <output-dir compare="Text">getCapital</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="library-metadata">
+ <test-case FilePath="library-metadata">
+ <compilation-unit name="functionDataset">
+ <output-dir compare="Text">functionDataset</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="library-metadata">
+ <compilation-unit name="libraryDataset">
+ <output-dir compare="Text">libraryDataset</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="library-metadata">
+ <compilation-unit name="dataverseDataset">
+ <output-dir compare="Text">dataverseDataset</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+</test-suite>
+
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.1.mgx.aql b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.1.mgx.aql
new file mode 100644
index 0000000..2d8a23e
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.1.mgx.aql
@@ -0,0 +1 @@
+stop -n asterix
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.2.mgx.aql b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.2.mgx.aql
new file mode 100644
index 0000000..97ad91e
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.2.mgx.aql
@@ -0,0 +1 @@
+install -n asterix -d externallibtest -l testlib -p ../asterix-external-data/target/testlib-zip-binary-assembly.zip
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.3.mgx.aql b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.3.mgx.aql
new file mode 100644
index 0000000..4e99f33
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.3.mgx.aql
@@ -0,0 +1 @@
+start -n asterix
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.4.query.aql b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.4.query.aql
new file mode 100644
index 0000000..5a46092
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/installLibrary/installLibrary.4.query.aql
@@ -0,0 +1,2 @@
+for $x in dataset Metadata.Library
+return $x
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.1.mgx.aql b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.1.mgx.aql
new file mode 100644
index 0000000..2d8a23e
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.1.mgx.aql
@@ -0,0 +1 @@
+stop -n asterix
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.2.mgx.aql b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.2.mgx.aql
new file mode 100644
index 0000000..1b5ae40
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.2.mgx.aql
@@ -0,0 +1 @@
+uninstall -n asterix -d externallibtest -l testlib
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.3.mgx.aql b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.3.mgx.aql
new file mode 100644
index 0000000..4e99f33
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.3.mgx.aql
@@ -0,0 +1 @@
+start -n asterix
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.4.query.aql b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.4.query.aql
new file mode 100644
index 0000000..5a46092
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/queries/asterix-lifecycle/uninstallLibrary/uninstallLibrary.4.query.aql
@@ -0,0 +1,2 @@
+for $x in dataset Metadata.Library
+return $x
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/results/asterix-lifecycle/installLibrary/installLibrary.1.adm b/asterix-installer/src/test/resources/integrationts/lifecycle/results/asterix-lifecycle/installLibrary/installLibrary.1.adm
new file mode 100644
index 0000000..a5d5c9b
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/results/asterix-lifecycle/installLibrary/installLibrary.1.adm
@@ -0,0 +1 @@
+{ "DataverseName": "externallibtest", "Name": "testlib", "Timestamp": "Wed Apr 24 17:25:25 PDT 2013" }
diff --git a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.1.ddl.aql b/asterix-installer/src/test/resources/integrationts/lifecycle/results/asterix-lifecycle/uninstallLibrary/uninstallLibrary.1.adm
similarity index 100%
copy from asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.1.ddl.aql
copy to asterix-installer/src/test/resources/integrationts/lifecycle/results/asterix-lifecycle/uninstallLibrary/uninstallLibrary.1.adm
diff --git a/asterix-installer/src/test/resources/integrationts/lifecycle/testsuite.xml b/asterix-installer/src/test/resources/integrationts/lifecycle/testsuite.xml
index f1949ca..969da43 100644
--- a/asterix-installer/src/test/resources/integrationts/lifecycle/testsuite.xml
+++ b/asterix-installer/src/test/resources/integrationts/lifecycle/testsuite.xml
@@ -19,6 +19,16 @@
<output-dir compare="Text">backupRestore</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="asterix-lifecycle">
+ <compilation-unit name="installLibrary">
+ <output-dir compare="Text">installLibrary</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="asterix-lifecycle">
+ <compilation-unit name="uninstallLibrary">
+ <output-dir compare="Text">uninstallLibrary</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
</test-suite>
diff --git a/asterix-metadata/pom.xml b/asterix-metadata/pom.xml
index 6b1e8cf..febe440 100644
--- a/asterix-metadata/pom.xml
+++ b/asterix-metadata/pom.xml
@@ -1,18 +1,14 @@
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<!-- ! Copyright 2009-2013 by The Regents of the University of California
+ ! Licensed under the Apache License, Version 2.0 (the "License"); ! you may
+ not use this file except in compliance with the License. ! you may obtain
+ a copy of the License from ! ! http://www.apache.org/licenses/LICENSE-2.0
+ ! ! Unless required by applicable law or agreed to in writing, software !
+ distributed under the License is distributed on an "AS IS" BASIS, ! WITHOUT
+ WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. ! See the
+ License for the specific language governing permissions and ! limitations
+ under the License. ! -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<artifactId>asterix</artifactId>
@@ -51,13 +47,13 @@
</dependency>
<dependency>
<groupId>edu.uci.ics.asterix</groupId>
- <artifactId>asterix-external-data</artifactId>
+ <artifactId>asterix-runtime</artifactId>
<version>0.8.1-SNAPSHOT</version>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>edu.uci.ics.asterix</groupId>
- <artifactId>asterix-runtime</artifactId>
+ <artifactId>asterix-events</artifactId>
<version>0.8.1-SNAPSHOT</version>
<scope>compile</scope>
</dependency>
@@ -80,6 +76,12 @@
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-storage-am-lsm-rtree</artifactId>
</dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-hdfs-core</artifactId>
+ <version>${hyracks.version}</version>
+ </dependency>
+
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java
index 91be90b..f78eb31 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java
@@ -27,9 +27,14 @@
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.Feed;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
import edu.uci.ics.asterix.metadata.entities.Function;
import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.metadata.entities.Library;
import edu.uci.ics.asterix.metadata.entities.NodeGroup;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
/**
* Caches metadata entities such that the MetadataManager does not have to
@@ -38,6 +43,7 @@
* simply ignored, i.e., updates are not not applied to the cache.
*/
public class MetadataCache {
+
// Key is dataverse name.
protected final Map<String, Dataverse> dataverses = new HashMap<String, Dataverse>();
// Key is dataverse name. Key of value map is dataset name.
@@ -52,6 +58,15 @@
protected final Map<FunctionSignature, Function> functions = new HashMap<FunctionSignature, Function>();
// Key is adapter dataverse. Key of value map is the adapter name
protected final Map<String, Map<String, DatasourceAdapter>> adapters = new HashMap<String, Map<String, DatasourceAdapter>>();
+ // Key is FeedId
+ protected final Map<FeedConnectionId, FeedActivity> feedActivity = new HashMap<FeedConnectionId, FeedActivity>();
+
+ // Key is DataverseName, Key of the value map is the Policy name
+ protected final Map<String, Map<String, FeedPolicy>> feedPolicies = new HashMap<String, Map<String, FeedPolicy>>();
+ // Key is library dataverse. Key of value map is the library name
+ protected final Map<String, Map<String, Library>> libraries = new HashMap<String, Map<String, Library>>();
+ // Key is library dataverse. Key of value map is the feed name
+ protected final Map<String, Map<String, Feed>> feeds = new HashMap<String, Map<String, Feed>>();
// Key is DataverseName, Key of the value map is the Policy name
protected final Map<String, Map<String, CompactionPolicy>> compactionPolicies = new HashMap<String, Map<String, CompactionPolicy>>();
@@ -89,15 +104,21 @@
synchronized (datatypes) {
synchronized (functions) {
synchronized (adapters) {
- synchronized (compactionPolicies) {
- dataverses.clear();
- nodeGroups.clear();
- datasets.clear();
- indexes.clear();
- datatypes.clear();
- functions.clear();
- adapters.clear();
- compactionPolicies.clear();
+ synchronized (feedActivity) {
+ synchronized (libraries) {
+ synchronized (compactionPolicies) {
+ dataverses.clear();
+ nodeGroups.clear();
+ datasets.clear();
+ indexes.clear();
+ datatypes.clear();
+ functions.clear();
+ adapters.clear();
+ feedActivity.clear();
+ libraries.clear();
+ compactionPolicies.clear();
+ }
+ }
}
}
}
@@ -212,22 +233,43 @@
synchronized (indexes) {
synchronized (datatypes) {
synchronized (functions) {
- synchronized (compactionPolicies) {
- datasets.remove(dataverse.getDataverseName());
- indexes.remove(dataverse.getDataverseName());
- datatypes.remove(dataverse.getDataverseName());
- adapters.remove(dataverse.getDataverseName());
- compactionPolicies.remove(dataverse.getDataverseName());
- List<FunctionSignature> markedFunctionsForRemoval = new ArrayList<FunctionSignature>();
- for (FunctionSignature signature : functions.keySet()) {
- if (signature.getNamespace().equals(dataverse.getDataverseName())) {
- markedFunctionsForRemoval.add(signature);
+ synchronized (adapters) {
+ synchronized (libraries) {
+ synchronized (feedActivity) {
+ synchronized (feeds) {
+ synchronized (compactionPolicies) {
+ datasets.remove(dataverse.getDataverseName());
+ indexes.remove(dataverse.getDataverseName());
+ datatypes.remove(dataverse.getDataverseName());
+ adapters.remove(dataverse.getDataverseName());
+ compactionPolicies.remove(dataverse.getDataverseName());
+ List<FunctionSignature> markedFunctionsForRemoval = new ArrayList<FunctionSignature>();
+ for (FunctionSignature signature : functions.keySet()) {
+ if (signature.getNamespace().equals(dataverse.getDataverseName())) {
+ markedFunctionsForRemoval.add(signature);
+ }
+ }
+ for (FunctionSignature signature : markedFunctionsForRemoval) {
+ functions.remove(signature);
+ }
+ List<FeedConnectionId> feedActivitiesMarkedForRemoval = new ArrayList<FeedConnectionId>();
+ for (FeedConnectionId fid : feedActivity.keySet()) {
+ if (fid.getDataverse().equals(dataverse.getDataverseName())) {
+ feedActivitiesMarkedForRemoval.add(fid);
+ }
+ }
+ for (FeedConnectionId fid : feedActivitiesMarkedForRemoval) {
+ feedActivity.remove(fid);
+ }
+
+ libraries.remove(dataverse.getDataverseName());
+ feeds.remove(dataverse.getDataverseName());
+
+ return dataverses.remove(dataverse.getDataverseName());
+ }
+ }
}
}
- for (FunctionSignature signature : markedFunctionsForRemoval) {
- functions.remove(signature);
- }
- return dataverses.remove(dataverse.getDataverseName());
}
}
}
@@ -409,6 +451,32 @@
}
}
+ public Object addFeedPolicyIfNotExists(FeedPolicy feedPolicy) {
+ synchronized (feedPolicy) {
+ Map<String, FeedPolicy> p = feedPolicies.get(feedPolicy.getDataverseName());
+ if (p == null) {
+ p = new HashMap<String, FeedPolicy>();
+ p.put(feedPolicy.getPolicyName(), feedPolicy);
+ feedPolicies.put(feedPolicy.getDataverseName(), p);
+ } else {
+ if (p.get(feedPolicy.getPolicyName()) == null) {
+ p.put(feedPolicy.getPolicyName(), feedPolicy);
+ }
+ }
+ return null;
+ }
+ }
+
+ public Object dropFeedPolicy(FeedPolicy feedPolicy) {
+ synchronized (feedPolicies) {
+ Map<String, FeedPolicy> p = feedPolicies.get(feedPolicy.getDataverseName());
+ if (p != null && p.get(feedPolicy.getPolicyName()) != null) {
+ return p.remove(feedPolicy).getPolicyName();
+ }
+ return null;
+ }
+ }
+
public Object addAdapterIfNotExists(DatasourceAdapter adapter) {
synchronized (adapters) {
Map<String, DatasourceAdapter> adaptersInDataverse = adapters.get(adapter.getAdapterIdentifier()
@@ -435,4 +503,61 @@
return null;
}
}
+
+ public Object addFeedActivityIfNotExists(FeedActivity fa) {
+ synchronized (feedActivity) {
+ FeedConnectionId fid = new FeedConnectionId(fa.getDataverseName(), fa.getFeedName(), fa.getDatasetName());
+ if (!feedActivity.containsKey(fid)) {
+ feedActivity.put(fid, fa);
+ }
+ }
+ return null;
+ }
+
+ public Object dropFeedActivity(FeedActivity fa) {
+ synchronized (feedActivity) {
+ FeedConnectionId fid = new FeedConnectionId(fa.getDataverseName(), fa.getFeedName(), fa.getDatasetName());
+ return feedActivity.remove(fid);
+ }
+ }
+
+ public Object addLibraryIfNotExists(Library library) {
+ synchronized (libraries) {
+ Map<String, Library> libsInDataverse = libraries.get(library.getDataverseName());
+ boolean needToAddd = (libsInDataverse == null || libsInDataverse.get(library.getName()) != null);
+ if (needToAddd) {
+ if (libsInDataverse == null) {
+ libsInDataverse = new HashMap<String, Library>();
+ libraries.put(library.getDataverseName(), libsInDataverse);
+ }
+ return libsInDataverse.put(library.getDataverseName(), library);
+ }
+ return null;
+ }
+ }
+
+ public Object dropLibrary(Library library) {
+ synchronized (libraries) {
+ Map<String, Library> librariesInDataverse = libraries.get(library.getDataverseName());
+ if (librariesInDataverse != null) {
+ return librariesInDataverse.remove(library.getName());
+ }
+ return null;
+ }
+ }
+
+ public Object addFeedIfNotExists(Feed feed) {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ public Object dropFeed(Feed feed) {
+ synchronized (feeds) {
+ Map<String, Feed> feedsInDataverse = feeds.get(feed.getDataverseName());
+ if (feedsInDataverse != null) {
+ return feedsInDataverse.remove(feed.getFeedName());
+ }
+ return null;
+ }
+ }
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataException.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataException.java
index b606839..e58466d 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataException.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataException.java
@@ -17,6 +17,7 @@
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+
public class MetadataException extends AsterixException {
private static final long serialVersionUID = 1L;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
index 46159ba..590890a 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
@@ -32,11 +32,18 @@
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.Feed;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
+import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
import edu.uci.ics.asterix.metadata.entities.Function;
import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.metadata.entities.Library;
import edu.uci.ics.asterix.metadata.entities.Node;
import edu.uci.ics.asterix.metadata.entities.NodeGroup;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
/**
* Provides access to Asterix metadata via remote methods to the metadata node.
@@ -71,6 +78,7 @@
* with transaction ids of regular jobs or other metadata transactions.
*/
public class MetadataManager implements IMetadataManager {
+
// Set in init().
public static MetadataManager INSTANCE;
private final MetadataCache cache = new MetadataCache();
@@ -78,6 +86,7 @@
private IMetadataNode metadataNode;
private final ReadWriteLock metadataLatch;
private final AsterixMetadataProperties metadataProperties;
+ private IHyracksClientConnection hcc;
public MetadataManager(IAsterixStateProxy proxy, AsterixMetadataProperties metadataProperties) {
if (proxy == null) {
@@ -96,10 +105,14 @@
if (metadataNode != null) {
return;
}
- metadataNode = proxy.getMetadataNode();
- if (metadataNode == null) {
- throw new Error("Failed to get the MetadataNode.\n" + "The MetadataNode was configured to run on NC: "
- + metadataProperties.getMetadataNodeName());
+ while (metadataNode == null) {
+ metadataNode = proxy.getMetadataNode();
+ try {
+ Thread.sleep(500);
+ } catch (InterruptedException ie) {
+ throw new RemoteException("Interrupted while waiting for obtaining handle to Metadata node " + "("
+ + metadataProperties.getMetadataNodeName() + ")");
+ }
}
}
}
@@ -555,6 +568,16 @@
}
@Override
+ public void addFeedPolicy(MetadataTransactionContext mdTxnCtx, FeedPolicy feedPolicy) throws MetadataException {
+ try {
+ metadataNode.addFeedPolicy(mdTxnCtx.getJobId(), feedPolicy);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ mdTxnCtx.addFeedPolicy(feedPolicy);
+ }
+
+ @Override
public void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws MetadataException {
try {
metadataNode.initializeDatasetIdFactory(ctx.getJobId());
@@ -610,6 +633,77 @@
}
@Override
+ public void registerFeedActivity(MetadataTransactionContext ctx, FeedConnectionId feedId, FeedActivity feedActivity)
+ throws MetadataException {
+ try {
+ metadataNode.registerFeedActivity(ctx.getJobId(), feedId, feedActivity);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ @Override
+ public FeedActivity getRecentActivityOnFeedConnection(MetadataTransactionContext ctx, FeedConnectionId feedId,
+ FeedActivityType... feedActivityTypes) throws MetadataException {
+
+ FeedActivity feedActivity = null;
+ try {
+ feedActivity = metadataNode.getRecentFeedActivity(ctx.getJobId(), feedId, feedActivityTypes);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ return feedActivity;
+ }
+
+ public void dropLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName)
+ throws MetadataException {
+ try {
+ metadataNode.dropLibrary(ctx.getJobId(), dataverseName, libraryName);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ ctx.dropLibrary(dataverseName, libraryName);
+ }
+
+ @Override
+ public List<Library> getDataverseLibraries(MetadataTransactionContext ctx, String dataverseName)
+ throws MetadataException {
+ List<Library> dataverseLibaries = null;
+ try {
+ // Assuming that the transaction can read its own writes on the
+ // metadata node.
+ dataverseLibaries = metadataNode.getDataverseLibraries(ctx.getJobId(), dataverseName);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ // Don't update the cache to avoid checking against the transaction's
+ // uncommitted functions.
+ return dataverseLibaries;
+ }
+
+ @Override
+ public void addLibrary(MetadataTransactionContext ctx, Library library) throws MetadataException {
+ try {
+ metadataNode.addLibrary(ctx.getJobId(), library);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ ctx.addLibrary(library);
+ }
+
+ @Override
+ public Library getLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName)
+ throws MetadataException, RemoteException {
+ Library library = null;
+ try {
+ library = metadataNode.getLibrary(ctx.getJobId(), dataverseName, libraryName);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ return library;
+ }
+
+ @Override
public void acquireWriteLatch() {
metadataLatch.writeLock().lock();
}
@@ -628,4 +722,72 @@
public void releaseReadLatch() {
metadataLatch.readLock().unlock();
}
+
+ @Override
+ public FeedPolicy getFeedPolicy(MetadataTransactionContext ctx, String dataverse, String policyName)
+ throws MetadataException {
+
+ FeedPolicy FeedPolicy = null;
+ try {
+ FeedPolicy = metadataNode.getFeedPolicy(ctx.getJobId(), dataverse, policyName);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ return FeedPolicy;
+ }
+
+ @Override
+ public List<FeedActivity> getActiveFeeds(MetadataTransactionContext ctx, String dataverse, String dataset)
+ throws MetadataException {
+ List<FeedActivity> feedActivities = null;
+ try {
+ feedActivities = metadataNode.getActiveFeeds(ctx.getJobId(), dataverse, dataset);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ return feedActivities;
+ }
+
+ @Override
+ public Feed getFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException {
+ Feed feed = null;
+ try {
+ feed = metadataNode.getFeed(ctx.getJobId(), dataverse, feedName);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ return feed;
+ }
+
+ @Override
+ public void dropFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException {
+ try {
+ metadataNode.dropFeed(ctx.getJobId(), dataverse, feedName);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ ctx.dropFeed(dataverse, feedName);
+ }
+
+ @Override
+ public void addFeed(MetadataTransactionContext ctx, Feed feed) throws MetadataException {
+ try {
+ metadataNode.addFeed(ctx.getJobId(), feed);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ ctx.addFeed(feed);
+ }
+
+ public List<FeedActivity> getConnectFeedActivitiesForFeed(MetadataTransactionContext ctx, String dataverse,
+ String feedName) throws MetadataException {
+ List<FeedActivity> feedActivities = null;
+ try {
+ feedActivities = metadataNode.getDatasetsServedByFeed(ctx.getJobId(), dataverse, feedName);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ return feedActivities;
+ }
+
}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
index a3f736e..3c7e4a3 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
@@ -17,7 +17,12 @@
import java.rmi.RemoteException;
import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
+import java.util.Map;
+import java.util.Set;
import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
@@ -42,9 +47,14 @@
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.Feed;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
+import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
import edu.uci.ics.asterix.metadata.entities.Function;
import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.Library;
import edu.uci.ics.asterix.metadata.entities.Node;
import edu.uci.ics.asterix.metadata.entities.NodeGroup;
import edu.uci.ics.asterix.metadata.entitytupletranslators.CompactionPolicyTupleTranslator;
@@ -52,10 +62,16 @@
import edu.uci.ics.asterix.metadata.entitytupletranslators.DatasourceAdapterTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.DatatypeTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.DataverseTupleTranslator;
+import edu.uci.ics.asterix.metadata.entitytupletranslators.FeedActivityTupleTranslator;
+import edu.uci.ics.asterix.metadata.entitytupletranslators.FeedPolicyTupleTranslator;
+import edu.uci.ics.asterix.metadata.entitytupletranslators.FeedTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.FunctionTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.IndexTupleTranslator;
+import edu.uci.ics.asterix.metadata.entitytupletranslators.LibraryTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.NodeGroupTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.NodeTupleTranslator;
+import edu.uci.ics.asterix.metadata.feeds.FeedActivityIdFactory;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
import edu.uci.ics.asterix.metadata.valueextractors.DatasetNameValueExtractor;
import edu.uci.ics.asterix.metadata.valueextractors.DatatypeNameValueExtractor;
import edu.uci.ics.asterix.metadata.valueextractors.MetadataEntityValueExtractor;
@@ -166,7 +182,7 @@
DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(true);
ITupleReference datasetTuple = tupleReaderWriter.getTupleFromMetadataEntity(dataset);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
- if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
+ if (dataset.getDatasetType() == DatasetType.INTERNAL) {
// Add the primary index for the dataset.
InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
Index primaryIndex = new Index(dataset.getDataverseName(), dataset.getDatasetName(),
@@ -305,13 +321,15 @@
@Override
public void dropDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
try {
- List<Dataset> dataverseDatasets;
+ List<Dataset> dataverseDatasets;
+ Dataset ds;
dataverseDatasets = getDataverseDatasets(jobId, dataverseName);
if (dataverseDatasets != null && dataverseDatasets.size() > 0) {
// Drop all datasets in this dataverse.
for (int i = 0; i < dataverseDatasets.size(); i++) {
- dropDataset(jobId, dataverseName, dataverseDatasets.get(i).getDatasetName());
+ ds = dataverseDatasets.get(i);
+ dropDataset(jobId, dataverseName, ds.getDatasetName());
}
}
List<Datatype> dataverseDatatypes;
@@ -345,6 +363,17 @@
}
}
+ List<Feed> dataverseFeeds;
+ Feed feed;
+ dataverseFeeds = getDataverseFeeds(jobId, dataverseName);
+ if (dataverseFeeds != null && dataverseFeeds.size() > 0) {
+ // Drop all datasets in this dataverse.
+ for (int i = 0; i < dataverseFeeds.size(); i++) {
+ feed = dataverseFeeds.get(i);
+ dropFeed(jobId, dataverseName, feed.getFeedName());
+ }
+ }
+
// Delete the dataverse entry from the 'dataverse' dataset.
ITupleReference searchKey = createTuple(dataverseName);
// As a side effect, acquires an S lock on the 'dataverse' dataset
@@ -388,7 +417,7 @@
}
// Delete entry from secondary index 'group'.
- if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
+ if (dataset.getDatasetType() == DatasetType.INTERNAL) {
InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
ITupleReference groupNameSearchKey = createTuple(id.getNodeGroupName(), dataverseName, datasetName);
// Searches the index for the tuple to be deleted. Acquires an S
@@ -416,7 +445,7 @@
}
// Delete entry(s) from the 'indexes' dataset.
- if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
+ if (dataset.getDatasetType() == DatasetType.INTERNAL) {
List<Index> datasetIndexes = getDatasetIndexes(jobId, dataverseName, datasetName);
if (datasetIndexes != null) {
for (Index index : datasetIndexes) {
@@ -424,6 +453,7 @@
}
}
}
+
} catch (Exception e) {
throw new MetadataException(e);
}
@@ -644,6 +674,34 @@
}
}
+ @Override
+ public List<Feed> getDataverseFeeds(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName);
+ FeedTupleTranslator tupleReaderWriter = new FeedTupleTranslator(false);
+ IValueExtractor<Feed> valueExtractor = new MetadataEntityValueExtractor<Feed>(tupleReaderWriter);
+ List<Feed> results = new ArrayList<Feed>();
+ searchIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, searchKey, valueExtractor, results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ public List<Library> getDataverseLibraries(JobId jobId, String dataverseName) throws MetadataException,
+ RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName);
+ LibraryTupleTranslator tupleReaderWriter = new LibraryTupleTranslator(false);
+ IValueExtractor<Library> valueExtractor = new MetadataEntityValueExtractor<Library>(tupleReaderWriter);
+ List<Library> results = new ArrayList<Library>();
+ searchIndex(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, searchKey, valueExtractor, results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
+
private List<Datatype> getDataverseDatatypes(JobId jobId, String dataverseName) throws MetadataException,
RemoteException {
try {
@@ -778,6 +836,7 @@
}
return results.get(0);
} catch (Exception e) {
+ e.printStackTrace();
throw new MetadataException(e);
}
}
@@ -839,9 +898,9 @@
+ functionSignature.getArity());
// Searches the index for the tuple to be deleted. Acquires an S
// lock on the 'function' dataset.
- ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET,
+ ITupleReference functionTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET,
searchKey);
- deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, datasetTuple);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
// TODO: Change this to be a BTree specific exception, e.g.,
// BTreeKeyDoesNotExistException.
@@ -1177,7 +1236,321 @@
}
@Override
+ public void addLibrary(JobId jobId, Library library) throws MetadataException, RemoteException {
+ try {
+ // Insert into the 'Library' dataset.
+ LibraryTupleTranslator tupleReaderWriter = new LibraryTupleTranslator(true);
+ ITupleReference libraryTuple = tupleReaderWriter.getTupleFromMetadataEntity(library);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, libraryTuple);
+
+ } catch (TreeIndexException e) {
+ throw new MetadataException("A library with this name " + library.getDataverseName()
+ + " already exists in dataverse '" + library.getDataverseName() + "'.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+
+ }
+
+ @Override
+ public void dropLibrary(JobId jobId, String dataverseName, String libraryName) throws MetadataException,
+ RemoteException {
+ Library library;
+ try {
+ library = getLibrary(jobId, dataverseName, libraryName);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ if (library == null) {
+ throw new MetadataException("Cannot drop library '" + library + "' because it doesn't exist.");
+ }
+ try {
+ // Delete entry from the 'Library' dataset.
+ ITupleReference searchKey = createTuple(dataverseName, libraryName);
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the 'Adapter' dataset.
+ ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, datasetTuple);
+
+ // TODO: Change this to be a BTree specific exception, e.g.,
+ // BTreeKeyDoesNotExistException.
+ } catch (TreeIndexException e) {
+ throw new MetadataException("Cannot drop library '" + libraryName, e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+
+ }
+
+ @Override
+ public Library getLibrary(JobId jobId, String dataverseName, String libraryName) throws MetadataException,
+ RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverseName, libraryName);
+ LibraryTupleTranslator tupleReaderWriter = new LibraryTupleTranslator(false);
+ List<Library> results = new ArrayList<Library>();
+ IValueExtractor<Library> valueExtractor = new MetadataEntityValueExtractor<Library>(tupleReaderWriter);
+ searchIndex(jobId, MetadataPrimaryIndexes.LIBRARY_DATASET, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results.get(0);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
+
public int getMostRecentDatasetId() throws MetadataException, RemoteException {
return DatasetIdFactory.getMostRecentDatasetId();
}
+
+ @Override
+ public void registerFeedActivity(JobId jobId, FeedConnectionId feedId, FeedActivity feedActivity)
+ throws MetadataException, RemoteException {
+ try {
+ if (!FeedActivityIdFactory.isInitialized()) {
+ initializeFeedActivityIdFactory(jobId);
+ }
+ feedActivity.setActivityId(FeedActivityIdFactory.generateFeedActivityId());
+ FeedActivityTupleTranslator tupleReaderWriter = new FeedActivityTupleTranslator(true);
+ ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(feedActivity);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET, tuple);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+
+ }
+
+ @Override
+ public FeedActivity getRecentFeedActivity(JobId jobId, FeedConnectionId feedId, FeedActivityType... activityType)
+ throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(feedId.getDataverse(), feedId.getFeedName(),
+ feedId.getDatasetName());
+ FeedActivityTupleTranslator tupleReaderWriter = new FeedActivityTupleTranslator(false);
+ List<FeedActivity> results = new ArrayList<FeedActivity>();
+ IValueExtractor<FeedActivity> valueExtractor = new MetadataEntityValueExtractor<FeedActivity>(
+ tupleReaderWriter);
+ searchIndex(jobId, MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET, searchKey, valueExtractor, results);
+ if (!results.isEmpty()) {
+ Collections.sort(results);
+ if (activityType == null) {
+ return results.get(0);
+ } else {
+ for (FeedActivity result : results) {
+ for (FeedActivityType ft : activityType) {
+ if (result.getActivityType().equals(ft)) {
+ return result;
+ }
+ }
+ }
+ }
+ }
+ return null;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ @Override
+ public void initializeFeedActivityIdFactory(JobId jobId) throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple();
+ FeedActivityTupleTranslator tupleReaderWriter = new FeedActivityTupleTranslator(true);
+ List<FeedActivity> results = new ArrayList<FeedActivity>();
+ IValueExtractor<FeedActivity> valueExtractor = new MetadataEntityValueExtractor<FeedActivity>(
+ tupleReaderWriter);
+ searchIndex(jobId, MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET, searchKey, valueExtractor, results);
+ int maxActivityId = 0;
+ for (FeedActivity fa : results) {
+ if (maxActivityId < fa.getActivityId()) {
+ maxActivityId = fa.getActivityId();
+ }
+ }
+ FeedActivityIdFactory.initialize(maxActivityId);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+
+ }
+
+ @Override
+ public void addFeedPolicy(JobId jobId, FeedPolicy feedPolicy) throws MetadataException, RemoteException {
+ try {
+ // Insert into the 'FeedPolicy' dataset.
+ FeedPolicyTupleTranslator tupleReaderWriter = new FeedPolicyTupleTranslator(true);
+ ITupleReference feedPolicyTuple = tupleReaderWriter.getTupleFromMetadataEntity(feedPolicy);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, feedPolicyTuple);
+
+ } catch (TreeIndexException e) {
+ throw new MetadataException("A feed policy with this name " + feedPolicy.getPolicyName()
+ + " already exists in dataverse '" + feedPolicy.getPolicyName() + "'.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+
+ }
+
+ @Override
+ public FeedPolicy getFeedPolicy(JobId jobId, String dataverse, String policyName) throws MetadataException,
+ RemoteException {
+
+ try {
+ ITupleReference searchKey = createTuple(dataverse, policyName);
+ FeedPolicyTupleTranslator tupleReaderWriter = new FeedPolicyTupleTranslator(false);
+ List<FeedPolicy> results = new ArrayList<FeedPolicy>();
+ IValueExtractor<FeedPolicy> valueExtractor = new MetadataEntityValueExtractor<FeedPolicy>(tupleReaderWriter);
+ searchIndex(jobId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, searchKey, valueExtractor, results);
+ if (!results.isEmpty()) {
+ return results.get(0);
+ }
+ return null;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ @Override
+ public List<FeedActivity> getActiveFeeds(JobId jobId, String dataverse, String dataset) throws MetadataException,
+ RemoteException {
+ List<FeedActivity> activeFeeds = new ArrayList<FeedActivity>();
+ Map<FeedConnectionId, FeedActivity> aFeeds = new HashMap<FeedConnectionId, FeedActivity>();
+ boolean invalidArgs = (dataverse == null && dataset != null);
+ if (invalidArgs) {
+ throw new MetadataException("Invalid arguments " + dataverse + " " + dataset);
+ }
+ try {
+ ITupleReference searchKey = createTuple();
+ FeedActivityTupleTranslator tupleReaderWriter = new FeedActivityTupleTranslator(true);
+ List<FeedActivity> results = new ArrayList<FeedActivity>();
+ IValueExtractor<FeedActivity> valueExtractor = new MetadataEntityValueExtractor<FeedActivity>(
+ tupleReaderWriter);
+ searchIndex(jobId, MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET, searchKey, valueExtractor, results);
+ Collections.sort(results); // recent activity first
+ FeedConnectionId fid = null;
+ Set<FeedConnectionId> terminatedFeeds = new HashSet<FeedConnectionId>();
+ for (FeedActivity fa : results) {
+ if (dataverse != null) {
+ if (dataset != null
+ && (!fa.getDataverseName().equals(dataverse) || !dataset.equals(fa.getDatasetName()))) {
+ continue;
+ }
+ }
+
+ fid = new FeedConnectionId(fa.getDataverseName(), fa.getFeedName(), fa.getDatasetName());
+ switch (fa.getActivityType()) {
+ case FEED_BEGIN:
+ if (!terminatedFeeds.contains(fid)) {
+ if (aFeeds.get(fid) == null || fa.getActivityId() > aFeeds.get(fid).getActivityId()) {
+ aFeeds.put(fid, fa);
+ }
+ }
+ break;
+ case FEED_END:
+ terminatedFeeds.add(fid);
+ break;
+ default: //ignore
+ }
+ }
+ for (FeedActivity f : aFeeds.values()) {
+ System.out.println("ACTIVE FEEDS " + f.getFeedName());
+ activeFeeds.add(f);
+ }
+ return activeFeeds;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ @Override
+ public void addFeed(JobId jobId, Feed feed) throws MetadataException, RemoteException {
+ try {
+ // Insert into the 'Feed' dataset.
+ FeedTupleTranslator tupleReaderWriter = new FeedTupleTranslator(true);
+ ITupleReference feedTuple = tupleReaderWriter.getTupleFromMetadataEntity(feed);
+ insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, feedTuple);
+
+ } catch (TreeIndexException e) {
+ throw new MetadataException("A feed with this name " + feed.getFeedName()
+ + " already exists in dataverse '" + feed.getDataverseName() + "'.", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ @Override
+ public Feed getFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverse, feedName);
+ FeedTupleTranslator tupleReaderWriter = new FeedTupleTranslator(false);
+ List<Feed> results = new ArrayList<Feed>();
+ IValueExtractor<Feed> valueExtractor = new MetadataEntityValueExtractor<Feed>(tupleReaderWriter);
+ searchIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, searchKey, valueExtractor, results);
+ if (!results.isEmpty()) {
+ return results.get(0);
+ }
+ return null;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ @Override
+ public void dropFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException {
+
+ try {
+ ITupleReference searchKey = createTuple(dataverse, feedName);
+ // Searches the index for the tuple to be deleted. Acquires an S
+ // lock on the 'nodegroup' dataset.
+ ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FEED_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FEED_DATASET, tuple);
+ // TODO: Change this to be a BTree specific exception, e.g.,
+ // BTreeKeyDoesNotExistException.
+ } catch (TreeIndexException e) {
+ throw new MetadataException("Cannot drop feed '" + feedName + "' because it doesn't exist", e);
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+
+ }
+
+ public List<FeedActivity> getDatasetsServedByFeed(JobId jobId, String dataverse, String feedName)
+ throws MetadataException, RemoteException {
+ List<FeedActivity> feedActivities = new ArrayList<FeedActivity>();
+ try {
+ ITupleReference searchKey = createTuple(dataverse, feedName);
+ FeedActivityTupleTranslator tupleReaderWriter = new FeedActivityTupleTranslator(false);
+ List<FeedActivity> results = new ArrayList<FeedActivity>();
+ IValueExtractor<FeedActivity> valueExtractor = new MetadataEntityValueExtractor<FeedActivity>(
+ tupleReaderWriter);
+ searchIndex(jobId, MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET, searchKey, valueExtractor, results);
+
+ if (!results.isEmpty()) {
+ Collections.sort(results); // most recent feed activity
+ Set<String> terminatedDatasets = new HashSet<String>();
+ Set<String> activeDatasets = new HashSet<String>();
+
+ for (FeedActivity result : results) {
+ switch (result.getFeedActivityType()) {
+ case FEED_BEGIN:
+ if (!terminatedDatasets.contains(result.getDatasetName())) {
+ feedActivities.add(result);
+ activeDatasets.add(result.getDatasetName());
+ }
+ break;
+ case FEED_END:
+ if (!activeDatasets.contains(result.getDatasetName())) {
+ terminatedDatasets.add(result.getDatasetName());
+ }
+ break;
+ }
+
+ }
+ }
+ return feedActivities;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
+
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java
index 3705fa1..1c990af 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java
@@ -16,19 +16,23 @@
package edu.uci.ics.asterix.metadata;
import java.util.ArrayList;
+import java.util.Map;
import edu.uci.ics.asterix.common.functions.FunctionSignature;
import edu.uci.ics.asterix.common.transactions.JobId;
-import edu.uci.ics.asterix.external.dataset.adapter.AdapterIdentifier;
import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
import edu.uci.ics.asterix.metadata.entities.CompactionPolicy;
import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.Feed;
+import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
import edu.uci.ics.asterix.metadata.entities.Function;
import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.metadata.entities.Library;
import edu.uci.ics.asterix.metadata.entities.NodeGroup;
+import edu.uci.ics.asterix.metadata.feeds.AdapterIdentifier;
/**
* Used to implement serializable transactions against the MetadataCache.
@@ -130,6 +134,11 @@
logAndApply(new MetadataLogicalOperation(dataverse, false));
}
+ public void addLibrary(Library library) {
+ droppedCache.dropLibrary(library);
+ logAndApply(new MetadataLogicalOperation(library, true));
+ }
+
public void dropDataDatatype(String dataverseName, String datatypeName) {
Datatype datatype = new Datatype(dataverseName, datatypeName, null, false);
droppedCache.addDatatypeIfNotExists(datatype);
@@ -156,6 +165,12 @@
logAndApply(new MetadataLogicalOperation(adapter, false));
}
+ public void dropLibrary(String dataverseName, String libraryName) {
+ Library library = new Library(dataverseName, libraryName);
+ droppedCache.addLibraryIfNotExists(library);
+ logAndApply(new MetadataLogicalOperation(library, false));
+ }
+
public void logAndApply(MetadataLogicalOperation op) {
opLog.add(op);
doOperation(op);
@@ -201,10 +216,29 @@
return opLog;
}
+ public void addFeedPolicy(FeedPolicy feedPolicy) {
+ droppedCache.dropFeedPolicy(feedPolicy);
+ logAndApply(new MetadataLogicalOperation(feedPolicy, true));
+
+ }
+
+ public void addFeed(Feed feed) {
+ droppedCache.dropFeed(feed);
+ logAndApply(new MetadataLogicalOperation(feed, true));
+
+ }
+
+ public void dropFeed(String dataverse, String feedName) {
+ Feed feed = new Feed(dataverse, feedName, null, null, null);
+ droppedCache.addFeedIfNotExists(feed);
+ logAndApply(new MetadataLogicalOperation(feed, false));
+ }
+
@Override
public void clear() {
super.clear();
droppedCache.clear();
opLog.clear();
}
+
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterEventsSubscriber.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterEventsSubscriber.java
new file mode 100644
index 0000000..049a45c
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterEventsSubscriber.java
@@ -0,0 +1,47 @@
+package edu.uci.ics.asterix.metadata.api;
+
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import java.util.Set;
+
+import edu.uci.ics.asterix.metadata.cluster.IClusterManagementWorkResponse;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties.State;
+
+public interface IClusterEventsSubscriber {
+
+ /**
+ * @param deadNodeIds
+ * @return
+ */
+ public Set<IClusterManagementWork> notifyNodeFailure(Set<String> deadNodeIds);
+
+ /**
+ * @param joinedNodeId
+ * @return
+ */
+ public Set<IClusterManagementWork> notifyNodeJoin(String joinedNodeId);
+
+ /**
+ * @param response
+ */
+ public void notifyRequestCompletion(IClusterManagementWorkResponse response);
+
+ /**
+ * @param previousState
+ * @param newState
+ */
+ public void notifyStateChange(State previousState, State newState);
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedMessage.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
similarity index 68%
copy from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedMessage.java
copy to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
index dcef2c8..65ac354 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedMessage.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
@@ -12,17 +12,18 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.external.feed.lifecycle;
+package edu.uci.ics.asterix.metadata.api;
-import java.io.Serializable;
+public interface IClusterManagementWork {
-public interface IFeedMessage extends Serializable {
-
- public enum MessageType {
- STOP,
- ALTER,
+ public enum WorkType {
+ ADD_NODE,
+ REMOVE_NODE
}
- public MessageType getMessageType();
+ public WorkType getClusterManagementWorkType();
+ public int getWorkId();
+
+ public IClusterEventsSubscriber getSourceSubscriber();
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManager.java
new file mode 100644
index 0000000..ea07a62
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManager.java
@@ -0,0 +1,38 @@
+package edu.uci.ics.asterix.metadata.api;
+
+import java.util.Set;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
+
+public interface IClusterManager {
+
+ /**
+ * @param node
+ * @throws AsterixException
+ */
+ public void addNode(Node node) throws AsterixException;
+
+ /**
+ * @param node
+ * @throws AsterixException
+ */
+ public void removeNode(Node node) throws AsterixException;
+
+ /**
+ * @param subscriber
+ */
+ public void registerSubscriber(IClusterEventsSubscriber subscriber);
+
+ /**
+ * @param sunscriber
+ * @return
+ */
+ public boolean deregisterSubscriber(IClusterEventsSubscriber sunscriber);
+
+ /**
+ * @return
+ */
+ public Set<IClusterEventsSubscriber> getRegisteredClusterEventSubscribers();
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
index 392c8a1..f23275c 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
@@ -8,7 +8,7 @@
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * WIThOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
@@ -27,10 +27,16 @@
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.Feed;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
+import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
import edu.uci.ics.asterix.metadata.entities.Function;
import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.metadata.entities.Library;
import edu.uci.ics.asterix.metadata.entities.Node;
import edu.uci.ics.asterix.metadata.entities.NodeGroup;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
/**
* A metadata manager provides user access to Asterix metadata (e.g., types,
@@ -458,6 +464,79 @@
public List<Function> getDataverseFunctions(MetadataTransactionContext ctx, String dataverseName)
throws MetadataException;
+ /**
+ * @param ctx
+ * @param feed
+ * @throws MetadataException
+ */
+ public void addFeed(MetadataTransactionContext ctx, Feed feed) throws MetadataException;
+
+ /**
+ * @param ctx
+ * @param dataverse
+ * @param feedName
+ * @return
+ * @throws MetadataException
+ */
+ public Feed getFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException;
+
+ /**
+ * @param ctx
+ * @param dataverse
+ * @param feedName
+ * @throws MetadataException
+ */
+ public void dropFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException;
+
+ /**
+ * @param ctx
+ * @param feedId
+ * @param feedActivity
+ * @throws MetadataException
+ */
+ public void registerFeedActivity(MetadataTransactionContext ctx, FeedConnectionId feedId, FeedActivity feedActivity)
+ throws MetadataException;
+
+ /**
+ * @param ctx
+ * @param dataverseName
+ * @param datasetName
+ * @return
+ * @throws MetadataException
+ */
+ public FeedActivity getRecentActivityOnFeedConnection(MetadataTransactionContext ctx, FeedConnectionId feedId,
+ FeedActivityType... activityTypeFilter) throws MetadataException;
+
+ /**
+ * @param ctx
+ * @param policy
+ * @throws MetadataException
+ */
+ public void addFeedPolicy(MetadataTransactionContext ctx, FeedPolicy policy) throws MetadataException;
+
+ /**
+ * @param ctx
+ * @param dataverse
+ * @param policyName
+ * @return
+ * @throws MetadataException
+ */
+ public FeedPolicy getFeedPolicy(MetadataTransactionContext ctx, String dataverse, String policyName)
+ throws MetadataException;
+
+ /**
+ * @param ctx
+ * @param dataverse
+ * @param dataset
+ * @return
+ * @throws MetadataException
+ */
+ public List<FeedActivity> getActiveFeeds(MetadataTransactionContext ctx, String dataverse, String dataset)
+ throws MetadataException;
+
+ public List<FeedActivity> getConnectFeedActivitiesForFeed(MetadataTransactionContext ctx, String dataverse,
+ String dataset) throws MetadataException;
+
public void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws MetadataException;
public int getMostRecentDatasetId() throws MetadataException;
@@ -470,4 +549,62 @@
public void releaseReadLatch();
+ /**
+ * 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 RemoteException
+ */
+ public void dropLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName)
+ throws MetadataException;
+
+ /**
+ * 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
+ * for example, if the library is already added.
+ * @throws RemoteException
+ */
+ public void addLibrary(MetadataTransactionContext ctx, Library library) throws MetadataException;
+
+ /**
+ * @param ctx
+ * MetadataTransactionContext of 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 RemoteException
+ */
+ public Library getLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName)
+ throws MetadataException, RemoteException;
+
+ /**
+ * Retireve libraries installed in a given dataverse.
+ *
+ * @param ctx
+ * MetadataTransactionContext of an active metadata transaction.
+ * @param dataverseName
+ * dataverse asociated with the library that is to be retrieved.
+ * @return Library
+ * @throws MetadataException
+ * @throws RemoteException
+ */
+ public List<Library> getDataverseLibraries(MetadataTransactionContext ctx, String dataverseName)
+ throws MetadataException;
+
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
index f27268f..8434847 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
@@ -29,10 +29,16 @@
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.Feed;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
+import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
import edu.uci.ics.asterix.metadata.entities.Function;
import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.metadata.entities.Library;
import edu.uci.ics.asterix.metadata.entities.Node;
import edu.uci.ics.asterix.metadata.entities.NodeGroup;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
/**
* A metadata node stores metadata in its local storage structures (currently
@@ -482,8 +488,19 @@
public void addCompactionPolicy(JobId jobId, CompactionPolicy compactionPolicy) throws MetadataException,
RemoteException;
+ public FeedActivity getRecentFeedActivity(JobId jobId, FeedConnectionId feedId,
+ FeedActivityType... feedActivityFilter) throws MetadataException, RemoteException;
+
/**
* @param jobId
+ * @throws MetadataException
+ * @throws RemoteException
+ */
+ public void initializeFeedActivityIdFactory(JobId jobId) throws MetadataException, RemoteException;
+
+ /**
+ *
+ * @param jobId
* @param dataverse
* @param policy
* @return
@@ -493,8 +510,165 @@
public CompactionPolicy getCompactionPolicy(JobId jobId, String dataverse, String policy) throws MetadataException,
RemoteException;
+ /**
+ *
+ * @param jobId
+ * @throws MetadataException
+ * @throws RemoteException
+ */
public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException;
+ /**
+ *
+ * @return
+ * @throws MetadataException
+ * @throws RemoteException
+ */
public int getMostRecentDatasetId() throws MetadataException, RemoteException;
+ /**
+ * @param jobId
+ * @param feed
+ * @throws MetadataException
+ * @throws RemoteException
+ */
+ public void addFeed(JobId jobId, Feed feed) throws MetadataException, RemoteException;
+
+ /**
+ * @param jobId
+ * @param dataverse
+ * @param feedName
+ * @return
+ * @throws MetadataException
+ * @throws RemoteException
+ */
+ public Feed getFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException;
+
+ /**
+ * @param jobId
+ * @param dataverse
+ * @param feedName
+ * @throws MetadataException
+ * @throws RemoteException
+ */
+ public void dropFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException;
+
+ /**
+ * @param jobId
+ * A globally unique id for an active metadata transaction.
+ * @param feedId
+ * A unique id for the feed
+ * @param feedActivity
+ */
+ public void registerFeedActivity(JobId jobId, FeedConnectionId feedId, FeedActivity feedActivity)
+ throws MetadataException, RemoteException;
+
+ /**
+ * @param jobId
+ * @param feedPolicy
+ * @throws MetadataException
+ * @throws RemoteException
+ */
+ public void addFeedPolicy(JobId jobId, FeedPolicy feedPolicy) throws MetadataException, RemoteException;
+
+ /**
+ * @param jobId
+ * @param dataverse
+ * @param policy
+ * @return
+ * @throws MetadataException
+ * @throws RemoteException
+ */
+ public FeedPolicy getFeedPolicy(JobId jobId, String dataverse, String policy) throws MetadataException,
+ RemoteException;
+
+ /**
+ * @param jobId
+ * @param dataverse
+ * @param dataset
+ * @return
+ * @throws MetadataException
+ * @throws RemoteException
+ */
+ public List<FeedActivity> getActiveFeeds(JobId jobId, String dataverse, String dataset) throws MetadataException,
+ RemoteException;
+
+ /**
+ * Removes a library , acquiring local locks on behalf of the given
+ * transaction id.
+ *
+ * @param txnId
+ * 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.
+ * @throws RemoteException
+ */
+ public void dropLibrary(JobId jobId, String dataverseName, String libraryName) throws MetadataException,
+ RemoteException;
+
+ /**
+ * Adds a library, acquiring local locks on behalf of the given
+ * transaction id.
+ *
+ * @param txnId
+ * A globally unique id for an active metadata transaction.
+ * @param library
+ * Library to be added
+ * @throws MetadataException
+ * for example, if the library is already added.
+ * @throws RemoteException
+ */
+ public void addLibrary(JobId jobId, Library library) throws MetadataException, RemoteException;
+
+ /**
+ * @param txnId
+ * 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 RemoteException
+ */
+ public Library getLibrary(JobId jobId, String dataverseName, String libraryName) throws MetadataException,
+ RemoteException;
+
+ /**
+ * Retireve libraries installed in a given dataverse.
+ *
+ * @param txnId
+ * A globally unique id for an active metadata transaction.
+ * @param dataverseName
+ * dataverse asociated with the library that is to be retrieved.
+ * @return Library
+ * @throws MetadataException
+ * @throws RemoteException
+ */
+ public List<Library> getDataverseLibraries(JobId jobId, String dataverseName) throws MetadataException,
+ RemoteException;
+
+ /**
+ * @param jobId
+ * @param dataverseName
+ * @return
+ * @throws MetadataException
+ * @throws RemoteException
+ */
+ public List<Feed> getDataverseFeeds(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
+
+ /**
+ * @param jobId
+ * @param dataverseName
+ * @param deedName
+ * @return
+ * @throws MetadataException
+ * @throws RemoteException
+ */
+ public List<FeedActivity> getDatasetsServedByFeed(JobId jobId, String dataverseName, String deedName)
+ throws MetadataException, RemoteException;
+
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
index 455715d..0b2a6c7 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -37,8 +37,6 @@
import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import edu.uci.ics.asterix.external.adapter.factory.IAdapterFactory;
-import edu.uci.ics.asterix.external.dataset.adapter.AdapterIdentifier;
import edu.uci.ics.asterix.metadata.IDatasetDetails;
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.MetadataManager;
@@ -51,12 +49,16 @@
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails.FileStructure;
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
import edu.uci.ics.asterix.metadata.entities.Node;
import edu.uci.ics.asterix.metadata.entities.NodeGroup;
+import edu.uci.ics.asterix.metadata.feeds.AdapterIdentifier;
+import edu.uci.ics.asterix.metadata.feeds.BuiltinFeedPolicies;
+import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
@@ -119,7 +121,10 @@
MetadataPrimaryIndexes.DATASET_DATASET, MetadataPrimaryIndexes.DATATYPE_DATASET,
MetadataPrimaryIndexes.INDEX_DATASET, MetadataPrimaryIndexes.NODE_DATASET,
MetadataPrimaryIndexes.NODEGROUP_DATASET, MetadataPrimaryIndexes.FUNCTION_DATASET,
- MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET };
+ MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, MetadataPrimaryIndexes.FEED_DATASET,
+ MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET, MetadataPrimaryIndexes.FEED_POLICY_DATASET,
+ MetadataPrimaryIndexes.LIBRARY_DATASET, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET };
+
secondaryIndexes = new IMetadataIndex[] { MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX,
MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX,
MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX };
@@ -171,6 +176,7 @@
insertNodes(mdTxnCtx);
insertInitialGroups(mdTxnCtx);
insertInitialAdapters(mdTxnCtx);
+ insertInitialFeedPolicies(mdTxnCtx);
insertInitialCompactionPolicies(mdTxnCtx);
if (LOGGER.isLoggable(Level.INFO)) {
@@ -310,7 +316,11 @@
"edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory",
"edu.uci.ics.asterix.external.adapter.factory.PullBasedTwitterAdapterFactory",
"edu.uci.ics.asterix.external.adapter.factory.RSSFeedAdapterFactory",
- "edu.uci.ics.asterix.external.adapter.factory.CNNFeedAdapterFactory", };
+ "edu.uci.ics.asterix.external.adapter.factory.CNNFeedAdapterFactory",
+ "edu.uci.ics.asterix.tools.external.data.RateControlledFileSystemBasedAdapterFactory",
+ "edu.uci.ics.asterix.tools.external.data.TwitterFirehoseFeedAdapterFactory",
+ "edu.uci.ics.asterix.tools.external.data.GenericSocketFeedAdapterFactory",
+ "edu.uci.ics.asterix.tools.external.data.SyntheticTwitterFeedAdapterFactory" };
DatasourceAdapter adapter;
for (String adapterClassName : builtInAdapterClassNames) {
adapter = getAdapter(adapterClassName);
@@ -318,6 +328,12 @@
}
}
+ private static void insertInitialFeedPolicies(MetadataTransactionContext mdTxnCtx) throws Exception {
+ for (FeedPolicy feedPolicy : BuiltinFeedPolicies.policies) {
+ MetadataManager.INSTANCE.addFeedPolicy(mdTxnCtx, feedPolicy);
+ }
+ }
+
private static void insertInitialCompactionPolicies(MetadataTransactionContext mdTxnCtx) throws Exception {
String[] builtInCompactionPolicyClassNames = new String[] {
"edu.uci.ics.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyFactory",
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataConstants.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataConstants.java
index 4c9b7d2..1957666 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataConstants.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataConstants.java
@@ -19,10 +19,10 @@
// Name of the dataverse the metadata lives in.
public final static String METADATA_DATAVERSE_NAME = "Metadata";
-
+
// Name of the node group where metadata is stored on.
public final static String METADATA_NODEGROUP_NAME = "MetadataGroup";
-
+
// Name of the default nodegroup where internal/feed datasets will be partitioned
// if an explicit nodegroup is not specified at the time of creation of a dataset
public static final String METADATA_DEFAULT_NODEGROUP_NAME = "DEFAULT_NG_ALL_NODES";
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
index f6bc9e6..68f9a74 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
@@ -32,6 +32,13 @@
public static IMetadataIndex NODE_DATASET;
public static IMetadataIndex NODEGROUP_DATASET;
public static IMetadataIndex FUNCTION_DATASET;
+ public static IMetadataIndex DATASOURCE_ADAPTER_DATASET;
+ public static IMetadataIndex LIBRARY_DATASET;
+ public static IMetadataIndex FEED_DATASET;
+ public static IMetadataIndex FEED_ACTIVITY_DATASET;
+ public static IMetadataIndex FEED_POLICY_DATASET;
+ public static IMetadataIndex COMPACTION_POLICY_DATASET;
+
public static final int METADATA_DATASET_ID = 0;
public static final int DATAVERSE_DATASET_ID = 1;
public static final int DATASET_DATASET_ID = 2;
@@ -41,11 +48,15 @@
public static final int NODEGROUP_DATASET_ID = 6;
public static final int FUNCTION_DATASET_ID = 7;
public static final int DATASOURCE_ADAPTER_DATASET_ID = 8;
- public static final int COMPACTION_POLICY_DATASET_ID = 9;
+
+ public static final int LIBRARY_DATASET_ID = 9;
+ public static final int FEED_DATASET_ID = 10;
+ public static final int FEED_ACTIVITY_DATASET_ID = 11;
+ public static final int FEED_POLICY_DATASET_ID = 12;
+ public static final int COMPACTION_POLICY_DATASET_ID = 13;
+
public static final int FIRST_AVAILABLE_USER_DATASET_ID = 100;
- public static IMetadataIndex DATASOURCE_ADAPTER_DATASET;
- public static IMetadataIndex COMPACTION_POLICY_DATASET;
/**
* Create all metadata primary index descriptors. MetadataRecordTypes must
@@ -95,6 +106,23 @@
MetadataRecordTypes.DATASOURCE_ADAPTER_RECORDTYPE, DATASOURCE_ADAPTER_DATASET_ID, true, new int[] { 0,
1 });
+ FEED_DATASET = new MetadataIndex("Feed", null, 3, new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
+ new String[] { "DataverseName", "FeedName" }, 0, MetadataRecordTypes.FEED_RECORDTYPE, FEED_DATASET_ID,
+ true, new int[] { 0, 1 });
+
+ FEED_ACTIVITY_DATASET = new MetadataIndex("FeedActivity", null, 5, new IAType[] { BuiltinType.ASTRING,
+ BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.AINT32 }, new String[] { "DataverseName",
+ "FeedName", "DatasetName", "ActivityId" }, 0, MetadataRecordTypes.FEED_ACTIVITY_RECORDTYPE,
+ FEED_ACTIVITY_DATASET_ID, true, new int[] { 0, 1, 2, 3 });
+
+ LIBRARY_DATASET = new MetadataIndex("Library", null, 3,
+ new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName", "Name" }, 0,
+ MetadataRecordTypes.LIBRARY_RECORDTYPE, LIBRARY_DATASET_ID, true, new int[] { 0, 1 });
+
+ FEED_POLICY_DATASET = new MetadataIndex("FeedPolicy", null, 3, new IAType[] { BuiltinType.ASTRING,
+ BuiltinType.ASTRING }, new String[] { "DataverseName", "PolicyName" }, 0,
+ MetadataRecordTypes.FEED_POLICY_RECORDTYPE, FEED_POLICY_DATASET_ID, true, new int[] { 0, 1 });
+
COMPACTION_POLICY_DATASET = new MetadataIndex("CompactionPolicy", null, 3, new IAType[] { BuiltinType.ASTRING,
BuiltinType.ASTRING }, new String[] { "DataverseName", "CompactionPolicy" }, 0,
MetadataRecordTypes.COMPACTION_POLICY_RECORDTYPE, COMPACTION_POLICY_DATASET_ID, true,
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
index 8452340..4af3640 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -48,6 +48,13 @@
public static ARecordType NODEGROUP_RECORDTYPE;
public static ARecordType FUNCTION_RECORDTYPE;
public static ARecordType DATASOURCE_ADAPTER_RECORDTYPE;
+ public static ARecordType FEED_RECORDTYPE;
+ public static ARecordType FEED_ADAPTOR_CONFIGURATION_RECORDTYPE;
+ public static ARecordType FEED_ACTIVITY_RECORDTYPE;
+ public static ARecordType FEED_POLICY_RECORDTYPE;
+ public static ARecordType POLICY_PARAMS_RECORDTYPE;
+ public static ARecordType FEED_ACTIVITY_DETAILS_RECORDTYPE;
+ public static ARecordType LIBRARY_RECORDTYPE;
public static ARecordType COMPACTION_POLICY_RECORDTYPE;
/**
@@ -58,6 +65,7 @@
// depend on other types being created first.
// These calls are one "dependency chain".
try {
+ POLICY_PARAMS_RECORDTYPE = createPropertiesRecordType();
DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE = createPropertiesRecordType();
COMPACTION_POLICY_PROPERTIES_RECORDTYPE = createPropertiesRecordType();
INTERNAL_DETAILS_RECORDTYPE = createInternalDetailsRecordType();
@@ -79,12 +87,33 @@
NODEGROUP_RECORDTYPE = createNodeGroupRecordType();
FUNCTION_RECORDTYPE = createFunctionRecordType();
DATASOURCE_ADAPTER_RECORDTYPE = createDatasourceAdapterRecordType();
+
+ FEED_RECORDTYPE = createFeedRecordType();
+ FEED_ADAPTOR_CONFIGURATION_RECORDTYPE = createPropertiesRecordType();
+ FEED_ACTIVITY_DETAILS_RECORDTYPE = createPropertiesRecordType();
+ FEED_ACTIVITY_RECORDTYPE = createFeedActivityRecordType();
+ FEED_POLICY_RECORDTYPE = createFeedPolicyRecordType();
+ LIBRARY_RECORDTYPE = createLibraryRecordType();
+
COMPACTION_POLICY_RECORDTYPE = createCompactionPolicyRecordType();
+
} catch (AsterixException e) {
throw new MetadataException(e);
}
}
+ public static final int FEED_POLICY_ARECORD_DATAVERSE_NAME_FIELD_INDEX = 0;
+ public static final int FEED_POLICY_ARECORD_POLICY_NAME_FIELD_INDEX = 1;
+ public static final int FEED_POLICY_ARECORD_DESCRIPTION_FIELD_INDEX = 2;
+ public static final int FEED_POLICY_ARECORD_PROPERTIES_FIELD_INDEX = 3;
+
+ private static ARecordType createFeedPolicyRecordType() throws AsterixException {
+ AUnorderedListType listPropertiesType = new AUnorderedListType(POLICY_PARAMS_RECORDTYPE, null);
+ String[] fieldNames = { "DataverseName", "PolicyName", "Description", "Properties" };
+ IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, listPropertiesType };
+ return new ARecordType("FeedPolicyRecordType", fieldNames, fieldTypes, true);
+ }
+
// Helper constants for accessing fields in an ARecord of type
// DataverseRecordType.
public static final int DATAVERSE_ARECORD_NAME_FIELD_INDEX = 0;
@@ -194,15 +223,14 @@
public static final int DATASET_ARECORD_DATASETTYPE_FIELD_INDEX = 3;
public static final int DATASET_ARECORD_INTERNALDETAILS_FIELD_INDEX = 4;
public static final int DATASET_ARECORD_EXTERNALDETAILS_FIELD_INDEX = 5;
- public static final int DATASET_ARECORD_FEEDDETAILS_FIELD_INDEX = 6;
- public static final int DATASET_ARECORD_HINTS_FIELD_INDEX = 7;
- public static final int DATASET_ARECORD_TIMESTAMP_FIELD_INDEX = 8;
- public static final int DATASET_ARECORD_DATASETID_FIELD_INDEX = 9;
- public static final int DATASET_ARECORD_PENDINGOP_FIELD_INDEX = 10;
+ public static final int DATASET_ARECORD_HINTS_FIELD_INDEX = 6;
+ public static final int DATASET_ARECORD_TIMESTAMP_FIELD_INDEX = 7;
+ public static final int DATASET_ARECORD_DATASETID_FIELD_INDEX = 8;
+ public static final int DATASET_ARECORD_PENDINGOP_FIELD_INDEX = 9;
private static final ARecordType createDatasetRecordType() throws AsterixException {
String[] fieldNames = { "DataverseName", "DatasetName", "DataTypeName", "DatasetType", "InternalDetails",
- "ExternalDetails", "FeedDetails", "Hints", "Timestamp", "DatasetId", "PendingOp" };
+ "ExternalDetails", "Hints", "Timestamp", "DatasetId", "PendingOp" };
List<IAType> internalRecordUnionList = new ArrayList<IAType>();
internalRecordUnionList.add(BuiltinType.ANULL);
@@ -214,16 +242,11 @@
externalRecordUnionList.add(EXTERNAL_DETAILS_RECORDTYPE);
AUnionType externalRecordUnion = new AUnionType(externalRecordUnionList, null);
- List<IAType> feedRecordUnionList = new ArrayList<IAType>();
- feedRecordUnionList.add(BuiltinType.ANULL);
- feedRecordUnionList.add(FEED_DETAILS_RECORDTYPE);
- AUnionType feedRecordUnion = new AUnionType(feedRecordUnionList, null);
-
AUnorderedListType unorderedListOfHintsType = new AUnorderedListType(DATASET_HINTS_RECORDTYPE, null);
IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
- internalRecordUnion, externalRecordUnion, feedRecordUnion, unorderedListOfHintsType,
- BuiltinType.ASTRING, BuiltinType.AINT32, BuiltinType.AINT32 };
+ internalRecordUnion, externalRecordUnion, unorderedListOfHintsType, BuiltinType.ASTRING,
+ BuiltinType.AINT32, BuiltinType.AINT32 };
return new ARecordType("DatasetRecordType", fieldNames, fieldTypes, true);
}
@@ -382,4 +405,58 @@
return new ARecordType("DatasourceAdapterRecordType", fieldNames, fieldTypes, true);
}
+ // Helper constants for accessing fields in an ARecord of type
+ // FeedActivityRecordType.
+ public static final int FEED_ACTIVITY_ARECORD_DATAVERSE_NAME_FIELD_INDEX = 0;
+ public static final int FEED_ACTIVITY_ARECORD_FEED_NAME_FIELD_INDEX = 1;
+ public static final int FEED_ACTIVITY_ARECORD_DATASET_NAME_FIELD_INDEX = 2;
+ public static final int FEED_ACTIVITY_ARECORD_ACTIVITY_ID_FIELD_INDEX = 3;
+ public static final int FEED_ACTIVITY_ARECORD_ACTIVITY_TYPE_FIELD_INDEX = 4;
+ public static final int FEED_ACTIVITY_ARECORD_DETAILS_FIELD_INDEX = 5;
+ public static final int FEED_ACTIVITY_ARECORD_LAST_UPDATE_TIMESTAMP_FIELD_INDEX = 6;
+
+ private static ARecordType createFeedActivityRecordType() throws AsterixException {
+ AUnorderedListType unorderedPropertyListType = new AUnorderedListType(FEED_ACTIVITY_DETAILS_RECORDTYPE, null);
+ String[] fieldNames = { "DataverseName", "FeedName", "DatasetName", "ActivityId", "ActivityType", "Details",
+ "Timestamp" };
+ IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.AINT32,
+ BuiltinType.ASTRING, unorderedPropertyListType, BuiltinType.ASTRING };
+ return new ARecordType("FeedActivityRecordType", fieldNames, fieldTypes, true);
+ }
+
+ public static final int FEED_ARECORD_DATAVERSE_NAME_FIELD_INDEX = 0;
+ public static final int FEED_ARECORD_FEED_NAME_FIELD_INDEX = 1;
+ public static final int FEED_ARECORD_ADAPTOR_NAME_FIELD_INDEX = 2;
+ public static final int FEED_ARECORD_ADAPTOR_CONFIGURATION_FIELD_INDEX = 3;
+ public static final int FEED_ARECORD_FUNCTION_FIELD_INDEX = 4;
+ public static final int FEED_ARECORD_TIMESTAMP_FIELD_INDEX = 5;
+
+ private static ARecordType createFeedRecordType() throws AsterixException {
+
+ AUnorderedListType unorderedAdaptorPropertyListType = new AUnorderedListType(
+ DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE, null);
+
+ List<IAType> feedFunctionUnionList = new ArrayList<IAType>();
+ feedFunctionUnionList.add(BuiltinType.ANULL);
+ feedFunctionUnionList.add(BuiltinType.ASTRING);
+ AUnionType feedFunctionUnion = new AUnionType(feedFunctionUnionList, null);
+
+ String[] fieldNames = { "DataverseName", "FeedName", "AdaptorName", "AdaptorConfiguration", "Function",
+ "Timestamp" };
+ IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
+ unorderedAdaptorPropertyListType, feedFunctionUnion, BuiltinType.ASTRING };
+
+ return new ARecordType("FeedRecordType", fieldNames, fieldTypes, true);
+
+ }
+
+ public static final int LIBRARY_ARECORD_DATAVERSENAME_FIELD_INDEX = 0;
+ public static final int LIBRARY_ARECORD_NAME_FIELD_INDEX = 1;
+ public static final int LIBRARY_ARECORD_TIMESTAMP_FIELD_INDEX = 2;
+
+ private static ARecordType createLibraryRecordType() throws AsterixException {
+ String[] fieldNames = { "DataverseName", "Name", "Timestamp" };
+ IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING };
+ return new ARecordType("LibraryRecordType", fieldNames, fieldTypes, true);
+ }
}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataSecondaryIndexes.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataSecondaryIndexes.java
index 315c5e9..c33c21f 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataSecondaryIndexes.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataSecondaryIndexes.java
@@ -56,5 +56,6 @@
BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName",
"NestedDatatypeName", "TopDatatypeName" }, 2, null, MetadataPrimaryIndexes.DATATYPE_DATASET_ID, false,
new int[] { 0, 2 });
+
}
}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/AbstractClusterManagementWork.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/AbstractClusterManagementWork.java
new file mode 100644
index 0000000..6948dbc
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/AbstractClusterManagementWork.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.cluster;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import edu.uci.ics.asterix.metadata.api.IClusterEventsSubscriber;
+import edu.uci.ics.asterix.metadata.api.IClusterManagementWork;
+
+public abstract class AbstractClusterManagementWork implements IClusterManagementWork {
+
+ protected final IClusterEventsSubscriber subscriber;
+
+ protected final int workId;
+
+ @Override
+ public int getWorkId() {
+ return workId;
+ }
+
+ public AbstractClusterManagementWork(IClusterEventsSubscriber subscriber) {
+ this.subscriber = subscriber;
+ this.workId = WorkIdGenerator.getNextWorkId();
+ }
+
+ private static class WorkIdGenerator {
+ private static AtomicInteger workId = new AtomicInteger(0);
+
+ public static int getNextWorkId() {
+ return workId.incrementAndGet();
+ }
+
+ }
+
+ @Override
+ public IClusterEventsSubscriber getSourceSubscriber() {
+ return subscriber;
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/AddNodeWork.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/AddNodeWork.java
new file mode 100644
index 0000000..68dcc4c
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/AddNodeWork.java
@@ -0,0 +1,27 @@
+package edu.uci.ics.asterix.metadata.cluster;
+
+import edu.uci.ics.asterix.metadata.api.IClusterEventsSubscriber;
+
+public class AddNodeWork extends AbstractClusterManagementWork {
+
+ private final int numberOfNodes;
+
+ @Override
+ public WorkType getClusterManagementWorkType() {
+ return WorkType.ADD_NODE;
+ }
+
+ public AddNodeWork(int numberOfNodes, IClusterEventsSubscriber subscriber) {
+ super(subscriber);
+ this.numberOfNodes = numberOfNodes;
+ }
+
+ public int getNumberOfNodes() {
+ return numberOfNodes;
+ }
+
+ @Override
+ public String toString() {
+ return WorkType.ADD_NODE + " " + numberOfNodes + " requested by " + subscriber;
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/AddNodeWorkResponse.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/AddNodeWorkResponse.java
new file mode 100644
index 0000000..40999f0
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/AddNodeWorkResponse.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.cluster;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class AddNodeWorkResponse extends ClusterManagementWorkResponse {
+
+ private final List<String> nodesToBeAdded;
+ private final List<String> nodesAdded;
+
+ public AddNodeWorkResponse(AddNodeWork w, List<String> nodesToBeAdded) {
+ super(w);
+ this.nodesToBeAdded = nodesToBeAdded;
+ this.nodesAdded = new ArrayList<String>();
+ }
+
+ public List<String> getNodesAdded() {
+ return nodesAdded;
+ }
+
+ public boolean updateProgress(String addedNode) {
+ nodesToBeAdded.remove(addedNode);
+ nodesAdded.add(addedNode);
+ return nodesToBeAdded.isEmpty();
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/ClusterManagementWorkResponse.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/ClusterManagementWorkResponse.java
new file mode 100644
index 0000000..d578a77
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/ClusterManagementWorkResponse.java
@@ -0,0 +1,31 @@
+package edu.uci.ics.asterix.metadata.cluster;
+
+import edu.uci.ics.asterix.metadata.api.IClusterManagementWork;
+
+public class ClusterManagementWorkResponse implements IClusterManagementWorkResponse {
+
+ protected final IClusterManagementWork work;
+
+ protected Status status;
+
+ public ClusterManagementWorkResponse(IClusterManagementWork w) {
+ this.work = w;
+ this.status = Status.IN_PROGRESS;
+ }
+
+ @Override
+ public IClusterManagementWork getWork() {
+ return work;
+ }
+
+ @Override
+ public Status getStatus() {
+ return status;
+ }
+
+ @Override
+ public void setStatus(Status status) {
+ this.status = status;
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/ClusterManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/ClusterManager.java
new file mode 100644
index 0000000..cefb431
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/ClusterManager.java
@@ -0,0 +1,169 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.cluster;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.Unmarshaller;
+
+import edu.uci.ics.asterix.common.config.AsterixExternalProperties;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
+import edu.uci.ics.asterix.event.model.AsterixInstance;
+import edu.uci.ics.asterix.event.schema.cluster.Cluster;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
+import edu.uci.ics.asterix.event.schema.pattern.Pattern;
+import edu.uci.ics.asterix.event.schema.pattern.Patterns;
+import edu.uci.ics.asterix.event.service.AsterixEventService;
+import edu.uci.ics.asterix.event.service.AsterixEventServiceUtil;
+import edu.uci.ics.asterix.event.service.ILookupService;
+import edu.uci.ics.asterix.event.service.ServiceProvider;
+import edu.uci.ics.asterix.event.util.PatternCreator;
+import edu.uci.ics.asterix.installer.schema.conf.Configuration;
+import edu.uci.ics.asterix.metadata.api.IClusterEventsSubscriber;
+import edu.uci.ics.asterix.metadata.api.IClusterManager;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
+
+public class ClusterManager implements IClusterManager {
+
+ private static final Logger LOGGER = Logger.getLogger(AsterixEventServiceClient.class.getName());
+
+ public static ClusterManager INSTANCE = new ClusterManager();
+
+ private static String eventsDir = System.getenv("user.dir") + File.separator + "eventrix";
+
+ private static AsterixEventServiceClient client;
+
+ private static ILookupService lookupService;
+
+ private static final Set<IClusterEventsSubscriber> eventSubscribers = new HashSet<IClusterEventsSubscriber>();
+
+ private ClusterManager() {
+ Cluster asterixCluster = AsterixClusterProperties.INSTANCE.getCluster();
+ String eventHome = asterixCluster == null ? null : asterixCluster.getWorkingDir().getDir();
+
+ if (asterixCluster != null) {
+ String asterixDir = System.getProperty("user.dir") + File.separator + "asterix";
+ File configFile = new File(System.getProperty("user.dir") + File.separator + "configuration.xml");
+ Configuration configuration = null;
+
+ try {
+ JAXBContext configCtx = JAXBContext.newInstance(Configuration.class);
+ Unmarshaller unmarshaller = configCtx.createUnmarshaller();
+ configuration = (Configuration) unmarshaller.unmarshal(configFile);
+ AsterixEventService.initialize(configuration, asterixDir, eventHome);
+ client = AsterixEventService.getAsterixEventServiceClient(AsterixClusterProperties.INSTANCE
+ .getCluster());
+
+ lookupService = ServiceProvider.INSTANCE.getLookupService();
+ if (!lookupService.isRunning(configuration)) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Lookup service not running. Starting lookup service ...");
+ }
+ lookupService.startService(configuration);
+ } else {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Lookup service running");
+ }
+ }
+
+ } catch (Exception e) {
+ throw new IllegalStateException("Unable to initialize cluster manager" + e);
+ }
+ }
+ }
+
+ @Override
+ public void addNode(Node node) throws AsterixException {
+ try {
+ Cluster cluster = AsterixClusterProperties.INSTANCE.getCluster();
+ List<Pattern> pattern = new ArrayList<Pattern>();
+ String asterixInstanceName = AsterixAppContextInfo.getInstance().getMetadataProperties().getInstanceName();
+ Patterns prepareNode = PatternCreator.INSTANCE.createPrepareNodePattern(asterixInstanceName,
+ AsterixClusterProperties.INSTANCE.getCluster(), node);
+ cluster.getNode().add(node);
+ client.submit(prepareNode);
+
+ AsterixExternalProperties externalProps = AsterixAppContextInfo.getInstance().getExternalProperties();
+ AsterixEventServiceUtil.poulateClusterEnvironmentProperties(cluster, externalProps.getCCJavaParams(),
+ externalProps.getNCJavaParams());
+
+ pattern.clear();
+ String ccHost = cluster.getMasterNode().getClusterIp();
+ String hostId = node.getId();
+ String nodeControllerId = asterixInstanceName + "_" + node.getId();
+ String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
+ Pattern startNC = PatternCreator.INSTANCE.createNCStartPattern(ccHost, hostId, nodeControllerId, iodevices);
+ pattern.add(startNC);
+ Patterns startNCPattern = new Patterns(pattern);
+ client.submit(startNCPattern);
+
+ removeNode(cluster.getSubstituteNodes().getNode(), node);
+
+ AsterixInstance instance = lookupService.getAsterixInstance(cluster.getInstanceName());
+ instance.getCluster().getNode().add(node);
+ removeNode(instance.getCluster().getSubstituteNodes().getNode(), node);
+ lookupService.updateAsterixInstance(instance);
+
+ } catch (Exception e) {
+ throw new AsterixException(e);
+ }
+
+ }
+
+ private void removeNode(List<Node> list, Node node) {
+ Node nodeToRemove = null;
+ for (Node n : list) {
+ if (n.getId().equals(node.getId())) {
+ nodeToRemove = n;
+ break;
+ }
+ }
+ if (nodeToRemove != null) {
+ boolean removed = list.remove(nodeToRemove);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("attempt to remove node :" + nodeToRemove + " successful " + removed);
+ }
+ }
+ }
+
+ @Override
+ public void removeNode(Node node) throws AsterixException {
+ // to be implemented later.
+ }
+
+ @Override
+ public void registerSubscriber(IClusterEventsSubscriber subscriber) {
+ eventSubscribers.add(subscriber);
+ }
+
+ @Override
+ public boolean deregisterSubscriber(IClusterEventsSubscriber subscriber) {
+ return eventSubscribers.remove(subscriber);
+ }
+
+ @Override
+ public Set<IClusterEventsSubscriber> getRegisteredClusterEventSubscribers() {
+ return eventSubscribers;
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/IClusterManagementWorkResponse.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/IClusterManagementWorkResponse.java
new file mode 100644
index 0000000..dfc88ac
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/IClusterManagementWorkResponse.java
@@ -0,0 +1,28 @@
+package edu.uci.ics.asterix.metadata.cluster;
+
+import edu.uci.ics.asterix.metadata.api.IClusterManagementWork;
+
+public interface IClusterManagementWorkResponse {
+
+ public enum Status {
+ IN_PROGRESS,
+ SUCCESS,
+ FAILURE
+ }
+
+ /**
+ * @return
+ */
+ public IClusterManagementWork getWork();
+
+ /**
+ * @return
+ */
+ public Status getStatus();
+
+ /**
+ * @param status
+ */
+ public void setStatus(Status status);
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/RemoveNodeWork.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/RemoveNodeWork.java
new file mode 100644
index 0000000..90683d1
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/RemoveNodeWork.java
@@ -0,0 +1,36 @@
+package edu.uci.ics.asterix.metadata.cluster;
+
+import java.util.Set;
+
+import edu.uci.ics.asterix.metadata.api.IClusterEventsSubscriber;
+
+public class RemoveNodeWork extends AbstractClusterManagementWork {
+
+ private final Set<String> nodesToBeRemoved;
+
+ @Override
+ public WorkType getClusterManagementWorkType() {
+ return WorkType.REMOVE_NODE;
+ }
+
+ public RemoveNodeWork(Set<String> nodesToBeRemoved, IClusterEventsSubscriber subscriber) {
+ super(subscriber);
+ this.nodesToBeRemoved = nodesToBeRemoved;
+ }
+
+ public Set<String> getNodesToBeRemoved() {
+ return nodesToBeRemoved;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append(WorkType.REMOVE_NODE);
+ for (String node : nodesToBeRemoved) {
+ builder.append(node + " ");
+ }
+ builder.append(" requested by " + subscriber);
+ return builder.toString();
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/RemoveNodeWorkResponse.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/RemoveNodeWorkResponse.java
new file mode 100644
index 0000000..58ea05e
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/RemoveNodeWorkResponse.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.cluster;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public class RemoveNodeWorkResponse extends ClusterManagementWorkResponse {
+
+ private Set<String> nodesToBeRemoved = new HashSet<String>();
+
+ public RemoveNodeWorkResponse(RemoveNodeWork w, Status status) {
+ super(w);
+ nodesToBeRemoved.addAll(w.getNodesToBeRemoved());
+ }
+
+ public boolean updateProgress(Set<String> failedNodeIds) {
+ nodesToBeRemoved.removeAll(failedNodeIds);
+ return nodesToBeRemoved.isEmpty();
+
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/DatasetHints.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/DatasetHints.java
index 0b7a7d9..2a80963 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/DatasetHints.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/DatasetHints.java
@@ -17,6 +17,8 @@
import java.util.HashSet;
import java.util.Set;
+import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
/**
@@ -51,6 +53,7 @@
private static Set<IHint> initHints() {
Set<IHint> hints = new HashSet<IHint>();
hints.add(new DatasetCardinalityHint());
+ hints.add(new DatasetNodegroupCardinalityHint());
return hints;
}
@@ -84,4 +87,43 @@
}
}
+
+ /**
+ * Hint representing the cardinality of nodes forming the nodegroup for the dataset.
+ */
+ public static class DatasetNodegroupCardinalityHint implements IHint {
+ public static final String NAME = "NODEGROUP_CARDINALITY";
+
+ public static final int DEFAULT = 1;
+
+ @Override
+ public String getName() {
+ return NAME;
+ }
+
+ @Override
+ public Pair<Boolean, String> validateValue(String value) {
+ boolean valid = true;
+ int intValue;
+ try {
+ intValue = Integer.parseInt(value);
+ if (intValue < 0) {
+ return new Pair<Boolean, String>(false, "Value must be >= 0");
+ }
+ int numNodesInCluster = AsterixAppContextInfo.getInstance().getMetadataProperties().getNodeNames()
+ .size();
+ if (numNodesInCluster < intValue) {
+ return new Pair<Boolean, String>(false,
+ "Value must be greater or equal to the existing number of nodes in cluster ("
+ + numNodesInCluster + ")");
+ }
+ } catch (NumberFormatException nfe) {
+ valid = false;
+ return new Pair<Boolean, String>(valid, "Inappropriate value");
+ }
+ return new Pair<Boolean, String>(true, null);
+ }
+
+ }
+
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/IHint.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/IHint.java
index d4a937d..519c831 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/IHint.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/IHint.java
@@ -16,6 +16,7 @@
import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+
/**
* Represents a hint provided as part of an AQL statement.
*/
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
index da15cfe..fa0311d 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
@@ -222,8 +222,8 @@
File relPathFile = new File(getRelativePath(datasetName + "_idx_" + targetIdxName));
Dataset dataset = findDataset(datasetName);
- if (dataset.getDatasetType() != DatasetType.INTERNAL & dataset.getDatasetType() != DatasetType.FEED) {
- throw new AlgebricksException("Not an internal or feed dataset");
+ if (dataset.getDatasetType() != DatasetType.INTERNAL) {
+ throw new AlgebricksException("Not an internal dataset");
}
InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
List<String> nodeGroup = findNodeGroupNodeNames(datasetDetails.getNodeGroupName());
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
index 166de00..874a838 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
@@ -15,16 +15,13 @@
package edu.uci.ics.asterix.metadata.declared;
-import java.io.IOException;
+import java.io.Serializable;
import java.util.ArrayList;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import java.util.Set;
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.metadata.entities.Dataset;
-import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
-import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
-import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.common.utils.ListSet;
@@ -32,7 +29,6 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
-import edu.uci.ics.hyracks.algebricks.core.algebra.properties.DefaultNodeGroupDomain;
import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
import edu.uci.ics.hyracks.algebricks.core.algebra.properties.INodeDomain;
@@ -44,142 +40,55 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
import edu.uci.ics.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
-public class AqlDataSource implements IDataSource<AqlSourceId> {
+public abstract class AqlDataSource implements IDataSource<AqlSourceId> {
private AqlSourceId id;
- private Dataset dataset;
- private IAType[] schemaTypes;
- private INodeDomain domain;
+ private String datasourceDataverse;
+ private String datasourceName;
private AqlDataSourceType datasourceType;
+ protected IAType[] schemaTypes;
+ protected INodeDomain domain;
+ private Map<String, Serializable> properties = new HashMap<String, Serializable>();
public enum AqlDataSourceType {
- INTERNAL,
- FEED,
- EXTERNAL,
- EXTERNAL_FEED
+ INTERNAL_DATASET,
+ EXTERNAL_DATASET,
+ FEED
}
- public AqlDataSource(AqlSourceId id, Dataset dataset, IAType itemType, AqlDataSourceType datasourceType)
- throws AlgebricksException {
+ public AqlDataSource(AqlSourceId id, String datasourceDataverse, String datasourceName, IAType itemType,
+ AqlDataSourceType datasourceType) throws AlgebricksException {
this.id = id;
- this.dataset = dataset;
+ this.datasourceDataverse = datasourceDataverse;
+ this.datasourceName = datasourceName;
this.datasourceType = datasourceType;
- try {
- switch (datasourceType) {
- case FEED:
- initFeedDataset(itemType, dataset);
- case INTERNAL: {
- initInternalDataset(itemType);
- break;
- }
- case EXTERNAL_FEED:
- case EXTERNAL: {
- initExternalDataset(itemType);
- break;
- }
- default: {
- throw new IllegalArgumentException();
- }
- }
- } catch (IOException e) {
- throw new AlgebricksException(e);
- }
}
- public AqlDataSource(AqlSourceId id, Dataset dataset, IAType itemType) throws AlgebricksException {
- this.id = id;
- this.dataset = dataset;
- try {
- switch (dataset.getDatasetType()) {
- case FEED:
- initFeedDataset(itemType, dataset);
- break;
- case INTERNAL:
- initInternalDataset(itemType);
- break;
- case EXTERNAL: {
- initExternalDataset(itemType);
- break;
- }
- default: {
- throw new IllegalArgumentException();
- }
- }
- } catch (IOException e) {
- throw new AlgebricksException(e);
- }
+ public String getDatasourceDataverse() {
+ return datasourceDataverse;
}
- // TODO: Seems like initFeedDataset() could simply call this method.
- private void initInternalDataset(IAType itemType) throws IOException {
- List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
- ARecordType recordType = (ARecordType) itemType;
- int n = partitioningKeys.size();
- schemaTypes = new IAType[n + 1];
- for (int i = 0; i < n; i++) {
- schemaTypes[i] = recordType.getFieldType(partitioningKeys.get(i));
- }
- schemaTypes[n] = itemType;
- domain = new DefaultNodeGroupDomain(DatasetUtils.getNodegroupName(dataset));
+ public String getDatasourceName() {
+ return datasourceName;
}
- private void initFeedDataset(IAType itemType, Dataset dataset) throws IOException {
- if (dataset.getDatasetDetails() instanceof ExternalDatasetDetails) {
- initExternalDataset(itemType);
- } else {
- List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
- int n = partitioningKeys.size();
- schemaTypes = new IAType[n + 1];
- ARecordType recordType = (ARecordType) itemType;
- for (int i = 0; i < n; i++) {
- schemaTypes[i] = recordType.getFieldType(partitioningKeys.get(i));
- }
- schemaTypes[n] = itemType;
- domain = new DefaultNodeGroupDomain(DatasetUtils.getNodegroupName(dataset));
- }
- }
+ public abstract IAType[] getSchemaTypes();
- private void initExternalDataset(IAType itemType) {
- schemaTypes = new IAType[1];
- schemaTypes[0] = itemType;
- INodeDomain domainForExternalData = new INodeDomain() {
- @Override
- public Integer cardinality() {
- return null;
- }
-
- @Override
- public boolean sameAs(INodeDomain domain) {
- return domain == this;
- }
- };
- domain = domainForExternalData;
- }
+ public abstract INodeDomain getDomain();
@Override
public AqlSourceId getId() {
return id;
}
- public Dataset getDataset() {
- return dataset;
- }
-
- @Override
- public IAType[] getSchemaTypes() {
- return schemaTypes;
- }
-
@Override
public String toString() {
return id.toString();
- // return "AqlDataSource(\"" + id.getDataverseName() + "/" +
- // id.getDatasetName() + "\")";
}
@Override
public IDataSourcePropertiesProvider getPropertiesProvider() {
- return new AqlDataSourcePartitioningProvider(dataset.getDatasetType(), domain);
+ return new AqlDataSourcePartitioningProvider(datasourceType, domain);
}
@Override
@@ -198,21 +107,24 @@
private INodeDomain domain;
- private DatasetType datasetType;
+ private AqlDataSourceType aqlDataSourceType;
- public AqlDataSourcePartitioningProvider(DatasetType datasetType, INodeDomain domain) {
- this.datasetType = datasetType;
+ public AqlDataSourcePartitioningProvider(AqlDataSourceType datasetSourceType, INodeDomain domain) {
+ this.aqlDataSourceType = datasetSourceType;
this.domain = domain;
}
@Override
public IPhysicalPropertiesVector computePropertiesVector(List<LogicalVariable> scanVariables) {
- switch (datasetType) {
- case EXTERNAL: {
+ IPhysicalPropertiesVector propsVector = null;
+
+ switch (aqlDataSourceType) {
+ case EXTERNAL_DATASET: {
IPartitioningProperty pp = new RandomPartitioningProperty(domain);
List<ILocalStructuralProperty> propsLocal = new ArrayList<ILocalStructuralProperty>();
- return new StructuralPropertiesVector(pp, propsLocal);
+ propsVector = new StructuralPropertiesVector(pp, propsLocal);
}
+
case FEED: {
int n = scanVariables.size();
IPartitioningProperty pp;
@@ -231,9 +143,11 @@
pp = new UnorderedPartitionedProperty(pvars, domain);
}
List<ILocalStructuralProperty> propsLocal = new ArrayList<ILocalStructuralProperty>();
- return new StructuralPropertiesVector(pp, propsLocal);
+ propsVector = new StructuralPropertiesVector(pp, propsLocal);
+ break;
}
- case INTERNAL: {
+
+ case INTERNAL_DATASET: {
int n = scanVariables.size();
IPartitioningProperty pp;
if (n < 2) {
@@ -254,12 +168,15 @@
for (int i = 0; i < n - 1; i++) {
propsLocal.add(new LocalOrderProperty(new OrderColumn(scanVariables.get(i), OrderKind.ASC)));
}
- return new StructuralPropertiesVector(pp, propsLocal);
+ propsVector = new StructuralPropertiesVector(pp, propsLocal);
}
+ break;
+
default: {
throw new IllegalArgumentException();
}
}
+ return propsVector;
}
}
@@ -268,4 +185,12 @@
return datasourceType;
}
+ public Map<String, Serializable> getProperties() {
+ return properties;
+ }
+
+ public void setProperties(Map<String, Serializable> properties) {
+ this.properties = properties;
+ }
+
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
index 8301f5d..46b8268 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
@@ -40,17 +40,6 @@
import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
import edu.uci.ics.asterix.common.transactions.JobId;
import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
-import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
-import edu.uci.ics.asterix.external.adapter.factory.IAdapterFactory;
-import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
-import edu.uci.ics.asterix.external.adapter.factory.ITypedDatasetAdapterFactory;
-import edu.uci.ics.asterix.external.data.operator.ExternalDataScanOperatorDescriptor;
-import edu.uci.ics.asterix.external.data.operator.FeedIntakeOperatorDescriptor;
-import edu.uci.ics.asterix.external.data.operator.FeedMessageOperatorDescriptor;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.ITypedDatasourceAdapter;
-import edu.uci.ics.asterix.external.feed.lifecycle.FeedId;
-import edu.uci.ics.asterix.external.feed.lifecycle.IFeedMessage;
import edu.uci.ics.asterix.formats.base.IDataFormat;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
@@ -59,14 +48,30 @@
import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
import edu.uci.ics.asterix.metadata.dataset.hints.DatasetHints.DatasetCardinalityHint;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.Datatype;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
-import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.Feed;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityDetails;
+import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
+import edu.uci.ics.asterix.metadata.feeds.BuiltinFeedPolicies;
+import edu.uci.ics.asterix.metadata.feeds.EndFeedMessage;
+import edu.uci.ics.asterix.metadata.feeds.ExternalDataScanOperatorDescriptor;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.metadata.feeds.FeedIntakeOperatorDescriptor;
+import edu.uci.ics.asterix.metadata.feeds.FeedMessageOperatorDescriptor;
+import edu.uci.ics.asterix.metadata.feeds.FeedUtil;
+import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory;
+import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory.SupportedOperation;
+import edu.uci.ics.asterix.metadata.feeds.IFeedMessage;
+import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
+import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.om.types.ARecordType;
@@ -110,7 +115,7 @@
import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import edu.uci.ics.hyracks.algebricks.runtime.operators.std.SinkWriterRuntimeFactory;
-import edu.uci.ics.hyracks.api.context.ICCContext;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
@@ -129,7 +134,6 @@
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
-import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
@@ -161,14 +165,14 @@
private boolean asyncResults;
private ResultSetId resultSetId;
private IResultSerializerFactoryProvider resultSerializerFactoryProvider;
+ private IHyracksClientConnection hcc;
private final Dataverse defaultDataverse;
private JobId jobId;
private final AsterixStorageProperties storageProperties;
- private static final Map<String, String> adapterFactoryMapping = initializeAdapterFactoryMapping();
- private static Scheduler hdfsScheduler;
+ public static final Map<String, String> adapterFactoryMapping = initializeAdapterFactoryMapping();
public String getPropertyValue(String propertyName) {
return config.get(propertyName);
@@ -190,16 +194,6 @@
this.defaultDataverse = defaultDataverse;
this.stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
this.storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
- ICCContext ccContext = AsterixAppContextInfo.getInstance().getCCApplicationContext().getCCContext();
- try {
- if (hdfsScheduler == null) {
- //set the singleton hdfs scheduler
- hdfsScheduler = new Scheduler(ccContext.getClusterControllerInfo().getClientNetAddress(), ccContext
- .getClusterControllerInfo().getClientNetPort());
- }
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
}
public void setJobId(JobId jobId) {
@@ -286,34 +280,20 @@
List<LogicalVariable> projectVariables, boolean projectPushed, IOperatorSchema opSchema,
IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
throws AlgebricksException {
- Dataset dataset;
try {
- dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataSource.getId().getDataverseName(), dataSource
- .getId().getDatasetName());
-
- if (dataset == null) {
- throw new AlgebricksException("Unknown dataset " + dataSource.getId().getDatasetName()
- + " in dataverse " + dataSource.getId().getDataverseName());
- }
- switch (dataset.getDatasetType()) {
+ switch (((AqlDataSource) dataSource).getDatasourceType()) {
case FEED:
- if (dataSource instanceof ExternalFeedDataSource) {
- return buildExternalDatasetScan(jobSpec, dataset, dataSource);
- } else {
- return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, dataset, dataSource,
- context, implConfig);
+ return buildFeedIntakeRuntime(jobSpec, dataSource);
+ case INTERNAL_DATASET:
+ return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, dataSource, context,
+ implConfig);
- }
- case INTERNAL: {
- return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, dataset, dataSource,
- context, implConfig);
- }
- case EXTERNAL: {
- return buildExternalDatasetScan(jobSpec, dataset, dataSource);
- }
- default: {
+ case EXTERNAL_DATASET:
+ return buildExternalDatasetScan(jobSpec, dataSource);
+
+ default:
throw new IllegalArgumentException();
- }
+
}
} catch (MetadataException e) {
throw new AlgebricksException(e);
@@ -322,27 +302,26 @@
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildInternalDatasetScan(JobSpecification jobSpec,
List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
- Dataset dataset, IDataSource<AqlSourceId> dataSource, JobGenContext context, Object implConfig)
- throws AlgebricksException, MetadataException {
+ IDataSource<AqlSourceId> dataSource, JobGenContext context, Object implConfig) throws AlgebricksException,
+ MetadataException {
AqlSourceId asid = dataSource.getId();
String dataverseName = asid.getDataverseName();
String datasetName = asid.getDatasetName();
Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, datasetName);
- return buildBtreeRuntime(jobSpec, outputVars, opSchema, typeEnv, context, false, dataset,
- primaryIndex.getIndexName(), null, null, true, true, implConfig);
+ return buildBtreeRuntime(jobSpec, outputVars, opSchema, typeEnv, context, false,
+ ((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null, true, true,
+ implConfig);
}
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDatasetScan(JobSpecification jobSpec,
- Dataset dataset, IDataSource<AqlSourceId> dataSource) throws AlgebricksException, MetadataException {
+ IDataSource<AqlSourceId> dataSource) throws AlgebricksException, MetadataException {
+ Dataset dataset = ((DatasetDataSource) dataSource).getDataset();
String itemTypeName = dataset.getItemTypeName();
IAType itemType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataset.getDataverseName(), itemTypeName)
.getDatatype();
- if (dataSource instanceof ExternalFeedDataSource) {
- return buildFeedIntakeRuntime(jobSpec, dataset);
- } else {
- return buildExternalDataScannerRuntime(jobSpec, itemType,
- (ExternalDatasetDetails) dataset.getDatasetDetails(), NonTaggedDataFormat.INSTANCE);
- }
+ return buildExternalDataScannerRuntime(jobSpec, itemType, (ExternalDatasetDetails) dataset.getDatasetDetails(),
+ NonTaggedDataFormat.INSTANCE);
+
}
@SuppressWarnings("rawtypes")
@@ -353,8 +332,7 @@
throw new AlgebricksException("Can only scan datasets of records.");
}
- IGenericDatasetAdapterFactory adapterFactory;
- IDatasourceAdapter adapter;
+ IAdapterFactory adapterFactory;
String adapterName;
DatasourceAdapter adapterEntity;
String adapterFactoryClassname;
@@ -364,17 +342,25 @@
adapterName);
if (adapterEntity != null) {
adapterFactoryClassname = adapterEntity.getClassname();
- adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+ adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
} else {
adapterFactoryClassname = adapterFactoryMapping.get(adapterName);
if (adapterFactoryClassname == null) {
throw new AlgebricksException(" Unknown adapter :" + adapterName);
}
- adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+ adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
}
- adapter = ((IGenericDatasetAdapterFactory) adapterFactory).createAdapter(
- wrapProperties(datasetDetails.getProperties()), itemType);
+ Map<String, String> configuration = datasetDetails.getProperties();
+
+ switch (adapterFactory.getAdapterType()) {
+ case GENERIC:
+ ((IGenericAdapterFactory) adapterFactory).configure(configuration, (ARecordType) itemType);
+ break;
+ case TYPED:
+ ((ITypedAdapterFactory) adapterFactory).configure(configuration);
+ break;
+ }
} catch (AlgebricksException ae) {
throw ae;
} catch (Exception e) {
@@ -382,21 +368,20 @@
throw new AlgebricksException("Unable to create adapter " + e);
}
- if (!(adapter.getAdapterType().equals(IDatasourceAdapter.AdapterType.READ) || adapter.getAdapterType().equals(
- IDatasourceAdapter.AdapterType.READ_WRITE))) {
+ if (!(adapterFactory.getSupportedOperations().equals(SupportedOperation.READ) || adapterFactory
+ .getSupportedOperations().equals(SupportedOperation.READ_WRITE))) {
throw new AlgebricksException("external dataset adapter does not support read operation");
}
- ARecordType rt = (ARecordType) itemType;
ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
- ExternalDataScanOperatorDescriptor dataScanner = new ExternalDataScanOperatorDescriptor(jobSpec,
- wrapPropertiesEmpty(datasetDetails.getProperties()), rt, scannerDesc, adapterFactory);
+ ExternalDataScanOperatorDescriptor dataScanner = new ExternalDataScanOperatorDescriptor(jobSpec, scannerDesc,
+ adapterFactory);
AlgebricksPartitionConstraint constraint;
try {
- constraint = adapter.getPartitionConstraint();
+ constraint = adapterFactory.getPartitionConstraint();
} catch (Exception e) {
throw new AlgebricksException(e);
}
@@ -428,80 +413,72 @@
@SuppressWarnings("rawtypes")
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildFeedIntakeRuntime(JobSpecification jobSpec,
- Dataset dataset) throws AlgebricksException {
+ IDataSource<AqlSourceId> dataSource) throws AlgebricksException {
- FeedDatasetDetails datasetDetails = (FeedDatasetDetails) dataset.getDatasetDetails();
- DatasourceAdapter adapterEntity;
- IDatasourceAdapter adapter;
- IAdapterFactory adapterFactory;
- IAType adapterOutputType;
- String adapterName;
- String adapterFactoryClassname;
-
- try {
- adapterName = datasetDetails.getAdapterFactory();
- adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
- adapterName);
- if (adapterEntity != null) {
- adapterFactoryClassname = adapterEntity.getClassname();
- adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
- } else {
- adapterFactoryClassname = adapterFactoryMapping.get(adapterName);
- if (adapterFactoryClassname != null) {
- } else {
- // adapterName has been provided as a fully qualified
- // classname
- adapterFactoryClassname = adapterName;
- }
- adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
- }
-
- if (adapterFactory instanceof ITypedDatasetAdapterFactory) {
- adapter = ((ITypedDatasetAdapterFactory) adapterFactory).createAdapter(wrapProperties(datasetDetails
- .getProperties()));
- adapterOutputType = ((ITypedDatasourceAdapter) adapter).getAdapterOutputType();
- } else if (adapterFactory instanceof IGenericDatasetAdapterFactory) {
- String outputTypeName = datasetDetails.getProperties().get(IGenericDatasetAdapterFactory.KEY_TYPE_NAME);
- adapterOutputType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataset.getDataverseName(),
- outputTypeName).getDatatype();
- adapter = ((IGenericDatasetAdapterFactory) adapterFactory).createAdapter(
- wrapProperties(datasetDetails.getProperties()), adapterOutputType);
- } else {
- throw new IllegalStateException(" Unknown factory type for " + adapterFactoryClassname);
- }
- } catch (AlgebricksException ae) {
- throw ae;
- } catch (Exception e) {
- e.printStackTrace();
- throw new AlgebricksException("unable to create adapter " + e);
- }
-
- ISerializerDeserializer payloadSerde = NonTaggedDataFormat.INSTANCE.getSerdeProvider()
- .getSerializerDeserializer(adapterOutputType);
- RecordDescriptor feedDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
-
- FeedIntakeOperatorDescriptor feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, new FeedId(
- dataset.getDataverseName(), dataset.getDatasetName()), adapterFactoryClassname,
- this.wrapPropertiesEmpty(datasetDetails.getProperties()), (ARecordType) adapterOutputType, feedDesc,
- adapterFactory);
-
+ FeedDataSource feedDataSource = (FeedDataSource) dataSource;
+ FeedIntakeOperatorDescriptor feedIngestor = null;
+ org.apache.commons.lang3.tuple.Pair<IAdapterFactory, ARecordType> factoryOutput = null;
AlgebricksPartitionConstraint constraint = null;
+
try {
- constraint = adapter.getPartitionConstraint();
+ factoryOutput = FeedUtil.getFeedFactoryAndOutput(feedDataSource.getFeed(), mdTxnCtx);
+ IAdapterFactory adapterFactory = factoryOutput.getLeft();
+ ARecordType adapterOutputType = factoryOutput.getRight();
+
+ ISerializerDeserializer payloadSerde = NonTaggedDataFormat.INSTANCE.getSerdeProvider()
+ .getSerializerDeserializer(adapterOutputType);
+ RecordDescriptor feedDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
+
+ FeedPolicy feedPolicy = (FeedPolicy) ((AqlDataSource) dataSource).getProperties().get(
+ BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY);
+
+ feedPolicy.getProperties().put(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY, feedPolicy.getPolicyName());
+ feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, new FeedConnectionId(
+ feedDataSource.getDatasourceDataverse(), feedDataSource.getDatasourceName(), feedDataSource
+ .getFeedConnectionId().getDatasetName()), adapterFactory, (ARecordType) adapterOutputType,
+ feedDesc, feedPolicy.getProperties());
+
+ constraint = factoryOutput.getLeft().getPartitionConstraint();
} catch (Exception e) {
throw new AlgebricksException(e);
}
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedIngestor, constraint);
}
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildFeedMessengerRuntime(
- AqlMetadataProvider metadataProvider, JobSpecification jobSpec, FeedDatasetDetails datasetDetails,
- String dataverse, String dataset, List<IFeedMessage> feedMessages) throws AlgebricksException {
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc = metadataProvider
- .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverse, dataset, dataset);
- FeedMessageOperatorDescriptor feedMessenger = new FeedMessageOperatorDescriptor(jobSpec, dataverse, dataset,
- feedMessages);
- return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedMessenger, spPc.second);
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildSendFeedMessageRuntime(
+ JobSpecification jobSpec, String dataverse, String feedName, String dataset, IFeedMessage feedMessage,
+ String[] locations) throws AlgebricksException {
+ AlgebricksPartitionConstraint partitionConstraint = new AlgebricksAbsolutePartitionConstraint(locations);
+ FeedMessageOperatorDescriptor feedMessenger = new FeedMessageOperatorDescriptor(jobSpec, dataverse, feedName,
+ dataset, feedMessage);
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedMessenger, partitionConstraint);
+ }
+
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDisconnectFeedMessengerRuntime(
+ JobSpecification jobSpec, String dataverse, String feedName, String dataset, FeedActivity feedActivity)
+ throws AlgebricksException {
+ List<String> feedLocations = new ArrayList<String>();
+ String[] ingestLocs = feedActivity.getFeedActivityDetails().get(FeedActivityDetails.INGEST_LOCATIONS)
+ .split(",");
+ String[] computeLocs = feedActivity.getFeedActivityDetails().get(FeedActivityDetails.COMPUTE_LOCATIONS)
+ .split(",");
+ String[] storageLocs = feedActivity.getFeedActivityDetails().get(FeedActivityDetails.STORAGE_LOCATIONS)
+ .split(",");
+
+ for (String loc : ingestLocs) {
+ feedLocations.add(loc);
+ }
+ for (String loc : computeLocs) {
+ feedLocations.add(loc);
+ }
+ for (String loc : storageLocs) {
+ feedLocations.add(loc);
+ }
+
+ FeedConnectionId feedId = new FeedConnectionId(dataverse, feedName, dataset);
+ String[] locations = feedLocations.toArray(new String[] {});
+ IFeedMessage feedMessage = new EndFeedMessage(feedId);
+ return buildSendFeedMessageRuntime(jobSpec, dataverse, feedName, dataset, feedMessage, locations);
}
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildBtreeRuntime(JobSpecification jobSpec,
@@ -704,10 +681,11 @@
public IDataSourceIndex<String, AqlSourceId> findDataSourceIndex(String indexId, AqlSourceId dataSourceId)
throws AlgebricksException {
AqlDataSource ads = findDataSource(dataSourceId);
- Dataset dataset = ads.getDataset();
- if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ if (ads.getDatasourceType() == AqlDataSourceType.EXTERNAL_DATASET) {
throw new AlgebricksException("No index for external dataset " + dataSourceId);
}
+ Dataset dataset = ((DatasetDataSource) ads).getDataset();
+
try {
String indexName = (String) indexId;
Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
@@ -735,25 +713,24 @@
}
String tName = dataset.getItemTypeName();
IAType itemType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, aqlId.getDataverseName(), tName).getDatatype();
- return new AqlDataSource(aqlId, dataset, itemType);
+ AqlDataSourceType datasourceType = dataset.getDatasetType().equals(DatasetType.EXTERNAL) ? AqlDataSourceType.EXTERNAL_DATASET
+ : AqlDataSourceType.INTERNAL_DATASET;
+ return new DatasetDataSource(aqlId, aqlId.getDataverseName(), aqlId.getDatasetName(), itemType, datasourceType);
}
@Override
public boolean scannerOperatorIsLeaf(IDataSource<AqlSourceId> dataSource) {
- AqlSourceId asid = dataSource.getId();
- String dataverseName = asid.getDataverseName();
- String datasetName = asid.getDatasetName();
- Dataset dataset = null;
- try {
- dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
- } catch (MetadataException e) {
- throw new IllegalStateException(e);
+ boolean result = false;
+ switch (((AqlDataSource) dataSource).getDatasourceType()) {
+ case INTERNAL_DATASET:
+ case EXTERNAL_DATASET:
+ result = ((DatasetDataSource) dataSource).getDataset().getDatasetType() == DatasetType.EXTERNAL;
+ break;
+ case FEED:
+ result = true;
+ break;
}
-
- if (dataset == null) {
- throw new IllegalArgumentException("Unknown dataset " + datasetName + " in dataverse " + dataverseName);
- }
- return dataset.getDatasetType() == DatasetType.EXTERNAL;
+ return result;
}
@Override
@@ -1419,8 +1396,8 @@
try {
File relPathFile = new File(getRelativePath(dataverseName, datasetName + "_idx_" + targetIdxName));
Dataset dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
- if (dataset.getDatasetType() != DatasetType.INTERNAL & dataset.getDatasetType() != DatasetType.FEED) {
- throw new AlgebricksException("Not an internal or feed dataset");
+ if (dataset.getDatasetType() != DatasetType.INTERNAL) {
+ throw new AlgebricksException("Not an internal dataset");
}
InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
List<String> nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, datasetDetails.getNodeGroupName())
@@ -1516,6 +1493,22 @@
return type.getDatatype();
}
+ public Feed findFeed(String dataverse, String feedName) throws AlgebricksException {
+ try {
+ return MetadataManager.INSTANCE.getFeed(mdTxnCtx, dataverse, feedName);
+ } catch (MetadataException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ public FeedPolicy findFeedPolicy(String dataverse, String policyName) throws AlgebricksException {
+ try {
+ return MetadataManager.INSTANCE.getFeedPolicy(mdTxnCtx, dataverse, policyName);
+ } catch (MetadataException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
public List<Index> getDatasetIndexes(String dataverseName, String datasetName) throws AlgebricksException {
try {
return MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
@@ -1554,8 +1547,8 @@
private Map<String, Object> wrapProperties(Map<String, String> properties) {
Map<String, Object> wrappedProperties = new HashMap<String, Object>();
wrappedProperties.putAll(properties);
- wrappedProperties.put(HDFSAdapterFactory.SCHEDULER, hdfsScheduler);
- wrappedProperties.put(HDFSAdapterFactory.CLUSTER_LOCATIONS, getClusterLocations());
+ //wrappedProperties.put(SCHEDULER, hdfsScheduler);
+ //wrappedProperties.put(CLUSTER_LOCATIONS, getClusterLocations());
return wrappedProperties;
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/DatasetDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/DatasetDataSource.java
new file mode 100644
index 0000000..cb1124f
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/DatasetDataSource.java
@@ -0,0 +1,98 @@
+package edu.uci.ics.asterix.metadata.declared;
+
+import java.io.IOException;
+import java.util.List;
+
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.DefaultNodeGroupDomain;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.INodeDomain;
+
+public class DatasetDataSource extends AqlDataSource {
+
+ private Dataset dataset;
+
+ public DatasetDataSource(AqlSourceId id, String datasourceDataverse, String datasourceName, IAType itemType,
+ AqlDataSourceType datasourceType) throws AlgebricksException {
+ super(id, datasourceDataverse, datasourceName, itemType, datasourceType);
+ MetadataTransactionContext ctx = null;
+ try {
+ ctx = MetadataManager.INSTANCE.beginTransaction();
+ dataset = MetadataManager.INSTANCE.getDataset(ctx, datasourceDataverse, datasourceName);
+ if (dataset == null) {
+ throw new AlgebricksException("Unknown dataset " + datasourceName + " in dataverse "
+ + datasourceDataverse);
+ }
+ MetadataManager.INSTANCE.commitTransaction(ctx);
+ switch (dataset.getDatasetType()) {
+ case INTERNAL:
+ initInternalDataset(itemType);
+ break;
+ case EXTERNAL:
+ initExternalDataset(itemType);
+ break;
+
+ }
+ } catch (Exception e) {
+ if (ctx != null) {
+ try {
+ MetadataManager.INSTANCE.abortTransaction(ctx);
+ } catch (Exception e2) {
+ e2.addSuppressed(e);
+ throw new IllegalStateException("Unable to abort " + e2.getMessage());
+ }
+ }
+
+ }
+
+ }
+
+ public Dataset getDataset() {
+ return dataset;
+ }
+
+ private void initInternalDataset(IAType itemType) throws IOException {
+ List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+ ARecordType recordType = (ARecordType) itemType;
+ int n = partitioningKeys.size();
+ schemaTypes = new IAType[n + 1];
+ for (int i = 0; i < n; i++) {
+ schemaTypes[i] = recordType.getFieldType(partitioningKeys.get(i));
+ }
+ schemaTypes[n] = itemType;
+ domain = new DefaultNodeGroupDomain(DatasetUtils.getNodegroupName(dataset));
+ }
+
+ private void initExternalDataset(IAType itemType) {
+ schemaTypes = new IAType[1];
+ schemaTypes[0] = itemType;
+ INodeDomain domainForExternalData = new INodeDomain() {
+ @Override
+ public Integer cardinality() {
+ return null;
+ }
+
+ @Override
+ public boolean sameAs(INodeDomain domain) {
+ return domain == this;
+ }
+ };
+ domain = domainForExternalData;
+ }
+
+ @Override
+ public IAType[] getSchemaTypes() {
+ return schemaTypes;
+ }
+
+ @Override
+ public INodeDomain getDomain() {
+ return domain;
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ExternalFeedDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ExternalFeedDataSource.java
deleted file mode 100644
index ed5df31..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ExternalFeedDataSource.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.metadata.declared;
-
-import edu.uci.ics.asterix.metadata.entities.Dataset;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-
-public class ExternalFeedDataSource extends AqlDataSource {
-
- public ExternalFeedDataSource(AqlSourceId id, Dataset dataset, IAType itemType) throws AlgebricksException {
- super(id, dataset, itemType);
- }
-
- public ExternalFeedDataSource(AqlSourceId id, Dataset dataset, IAType itemType, AqlDataSourceType dataSourceType)
- throws AlgebricksException {
- super(id, dataset, itemType, dataSourceType);
- }
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FeedDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FeedDataSource.java
new file mode 100644
index 0000000..fc5d9ca
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FeedDataSource.java
@@ -0,0 +1,91 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.declared;
+
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.entities.Feed;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.INodeDomain;
+
+public class FeedDataSource extends AqlDataSource {
+
+ private Feed feed;
+ private final FeedConnectionId feedConnectionId;
+
+ public FeedDataSource(AqlSourceId id, FeedConnectionId feedId, IAType itemType, AqlDataSourceType dataSourceType)
+ throws AlgebricksException {
+ super(id, feedId.getDataverse(), feedId.getFeedName(), itemType, dataSourceType);
+ this.feedConnectionId = feedId;
+ feed = null;
+ MetadataTransactionContext ctx = null;
+ try {
+ ctx = MetadataManager.INSTANCE.beginTransaction();
+ feed = MetadataManager.INSTANCE.getFeed(ctx, feedId.getDataverse(), feedId.getFeedName());
+ if (feed == null) {
+ throw new AlgebricksException("Unknown feed " + feedId);
+ }
+ MetadataManager.INSTANCE.commitTransaction(ctx);
+ initFeedDataSource(itemType);
+ } catch (Exception e) {
+ if (ctx != null) {
+ try {
+ MetadataManager.INSTANCE.abortTransaction(ctx);
+ } catch (Exception e2) {
+ e2.addSuppressed(e);
+ throw new IllegalStateException("Unable to abort " + e2.getMessage());
+ }
+ }
+
+ }
+ }
+
+ public Feed getFeed() {
+ return feed;
+ }
+
+ @Override
+ public IAType[] getSchemaTypes() {
+ return schemaTypes;
+ }
+
+ @Override
+ public INodeDomain getDomain() {
+ return domain;
+ }
+
+ public FeedConnectionId getFeedConnectionId() {
+ return feedConnectionId;
+ }
+
+ private void initFeedDataSource(IAType itemType) {
+ schemaTypes = new IAType[1];
+ schemaTypes[0] = itemType;
+ INodeDomain domainForExternalData = new INodeDomain() {
+ @Override
+ public Integer cardinality() {
+ return null;
+ }
+
+ @Override
+ public boolean sameAs(INodeDomain domain) {
+ return domain == this;
+ }
+ };
+ domain = domainForExternalData;
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FileSplitDataSink.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FileSplitDataSink.java
index 3e61ec5..0827f09 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FileSplitDataSink.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FileSplitDataSink.java
@@ -18,6 +18,7 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSink;
import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+
public class FileSplitDataSink implements IDataSink {
private FileSplitSinkId id;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FileSplitSinkId.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FileSplitSinkId.java
index 5801002..9ca7291 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FileSplitSinkId.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FileSplitSinkId.java
@@ -17,6 +17,7 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+
public class FileSplitSinkId {
private FileSplit fileSplit;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetDataSink.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetDataSink.java
index 3fd9285..50c6225 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetDataSink.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetDataSink.java
@@ -20,6 +20,7 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ResultSetDomain;
+
public class ResultSetDataSink implements IDataSink {
private ResultSetSinkId id;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetSinkId.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetSinkId.java
index 99d026b..4fa3907 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetSinkId.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetSinkId.java
@@ -16,6 +16,7 @@
import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+
public class ResultSetSinkId {
private final ResultSetId resultSetId;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java
index 7455cec..09db248 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java
@@ -117,4 +117,9 @@
}
return true;
}
+
+ @Override
+ public String toString() {
+ return dataverseName + "." + datasetName;
+ }
}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/DatasourceAdapter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/DatasourceAdapter.java
index 3c72084..621933f 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/DatasourceAdapter.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/DatasourceAdapter.java
@@ -14,12 +14,14 @@
*/
package edu.uci.ics.asterix.metadata.entities;
-import edu.uci.ics.asterix.external.dataset.adapter.AdapterIdentifier;
import edu.uci.ics.asterix.metadata.MetadataCache;
import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
+import edu.uci.ics.asterix.metadata.feeds.AdapterIdentifier;
public class DatasourceAdapter implements IMetadataEntity {
+ private static final long serialVersionUID = 1L;
+
public enum AdapterType {
INTERNAL,
EXTERNAL
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Feed.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Feed.java
new file mode 100644
index 0000000..5d1704e
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Feed.java
@@ -0,0 +1,95 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.metadata.entities;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.asterix.metadata.IDatasetDetails;
+import edu.uci.ics.asterix.metadata.MetadataCache;
+import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
+
+/**
+ * Metadata describing a feed.
+ */
+public class Feed implements IMetadataEntity {
+
+ private static final long serialVersionUID = 1L;
+
+ private final String dataverseName;
+ private final String feedName;
+ private final String adaptorName;
+ private final Map<String, String> adaptorConfiguration;
+ private final FunctionSignature appliedFunction;
+
+ public Feed(String dataverseName, String datasetName, String adaptorName, Map<String, String> adaptorConfiguration,
+ FunctionSignature appliedFunction) {
+ this.dataverseName = dataverseName;
+ this.feedName = datasetName;
+ this.adaptorName = adaptorName;
+ this.adaptorConfiguration = adaptorConfiguration;
+ this.appliedFunction = appliedFunction;
+ }
+
+ public String getDataverseName() {
+ return dataverseName;
+ }
+
+ public String getFeedName() {
+ return feedName;
+ }
+
+ public String getAdaptorName() {
+ return adaptorName;
+ }
+
+ public Map<String, String> getAdaptorConfiguration() {
+ return adaptorConfiguration;
+ }
+
+ public FunctionSignature getAppliedFunction() {
+ return appliedFunction;
+ }
+
+ @Override
+ public Object addToCache(MetadataCache cache) {
+ return cache.addFeedIfNotExists(this);
+ }
+
+ @Override
+ public Object dropFromCache(MetadataCache cache) {
+ return cache.dropFeed(this);
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (this == other) {
+ return true;
+ }
+ if (!(other instanceof Feed)) {
+ return false;
+ }
+ Feed otherDataset = (Feed) other;
+ if (!otherDataset.dataverseName.equals(dataverseName)) {
+ return false;
+ }
+ if (!otherDataset.feedName.equals(feedName)) {
+ return false;
+ }
+ return true;
+ }
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedActivity.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedActivity.java
new file mode 100644
index 0000000..679276f
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedActivity.java
@@ -0,0 +1,175 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.metadata.entities;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.metadata.MetadataCache;
+import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
+
+/**
+ * Metadata describing a feed activity record.
+ */
+public class FeedActivity implements IMetadataEntity, Comparable<FeedActivity> {
+
+ private static final long serialVersionUID = 1L;
+
+ private int activityId;
+
+ private final String dataverseName;
+ private final String datasetName;
+ private final String feedName;
+
+ private String lastUpdatedTimestamp;
+ private FeedActivityType activityType;
+ private Map<String, String> feedActivityDetails;
+
+ public static enum FeedActivityType {
+ FEED_BEGIN,
+ FEED_FAILURE,
+ FEED_END
+ }
+
+ public static class FeedActivityDetails {
+ public static final String COMPUTE_LOCATIONS = "compute-locations";
+ public static final String INGEST_LOCATIONS = "ingest-locations";
+ public static final String STORAGE_LOCATIONS = "storage-locations";
+ public static final String TOTAL_INGESTED = "total-ingested";
+ public static final String INGESTION_RATE = "ingestion-rate";
+ public static final String EXCEPTION_LOCATION = "exception-location";
+ public static final String EXCEPTION_MESSAGE = "exception-message";
+ public static final String FEED_POLICY_NAME = "feed-policy-name";
+ public static final String SUPER_FEED_MANAGER_HOST = "super-feed-manager-host";
+ public static final String SUPER_FEED_MANAGER_PORT = "super-feed-manager-port";
+ public static final String FEED_NODE_FAILURE = "feed-node-failure";
+
+ }
+
+ public FeedActivity(String dataverseName, String feedName, String datasetName, FeedActivityType feedActivityType,
+ Map<String, String> feedActivityDetails) {
+ this.dataverseName = dataverseName;
+ this.feedName = feedName;
+ this.datasetName = datasetName;
+ this.activityType = feedActivityType;
+ this.feedActivityDetails = feedActivityDetails;
+ }
+
+ public String getDataverseName() {
+ return dataverseName;
+ }
+
+ public String getDatasetName() {
+ return datasetName;
+ }
+
+ public String getFeedName() {
+ return feedName;
+ }
+
+ @Override
+ public Object addToCache(MetadataCache cache) {
+ return cache.addFeedActivityIfNotExists(this);
+ }
+
+ @Override
+ public Object dropFromCache(MetadataCache cache) {
+ return cache.dropFeedActivity(this);
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (this == other) {
+ return true;
+ }
+ if (!(other instanceof FeedActivity)) {
+ return false;
+ }
+
+ if (!((FeedActivity) other).dataverseName.equals(dataverseName)) {
+ return false;
+ }
+ if (!((FeedActivity) other).datasetName.equals(datasetName)) {
+ return false;
+ }
+ if (!((FeedActivity) other).getFeedName().equals(feedName)) {
+ return false;
+ }
+ if (!((FeedActivity) other).getFeedActivityType().equals(activityType)) {
+ return false;
+ }
+ if (((FeedActivity) other).getActivityId() != (activityId)) {
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ return toString().hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return dataverseName + "." + feedName + " --> " + datasetName + " " + activityType + " " + activityId;
+ }
+
+ public FeedActivityType getFeedActivityType() {
+ return activityType;
+ }
+
+ public void setFeedActivityType(FeedActivityType feedActivityType) {
+ this.activityType = feedActivityType;
+ }
+
+ public String getLastUpdatedTimestamp() {
+ return lastUpdatedTimestamp;
+ }
+
+ public void setLastUpdatedTimestamp(String lastUpdatedTimestamp) {
+ this.lastUpdatedTimestamp = lastUpdatedTimestamp;
+ }
+
+ public int getActivityId() {
+ return activityId;
+ }
+
+ public void setActivityId(int activityId) {
+ this.activityId = activityId;
+ }
+
+ public Map<String, String> getFeedActivityDetails() {
+ return feedActivityDetails;
+ }
+
+ public void setFeedActivityDetails(Map<String, String> feedActivityDetails) {
+ this.feedActivityDetails = feedActivityDetails;
+ }
+
+ public FeedActivityType getActivityType() {
+ return activityType;
+ }
+
+ public void setActivityType(FeedActivityType activityType) {
+ this.activityType = activityType;
+ }
+
+ @Override
+ public int compareTo(FeedActivity o) {
+ return o.getActivityId() - this.activityId;
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java
deleted file mode 100644
index 5058e24..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java
+++ /dev/null
@@ -1,215 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.metadata.entities;
-
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import edu.uci.ics.asterix.builders.IARecordBuilder;
-import edu.uci.ics.asterix.builders.OrderedListBuilder;
-import edu.uci.ics.asterix.builders.RecordBuilder;
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.functions.FunctionSignature;
-import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import edu.uci.ics.asterix.metadata.bootstrap.MetadataRecordTypes;
-import edu.uci.ics.asterix.om.base.AMutableString;
-import edu.uci.ics.asterix.om.base.AString;
-import edu.uci.ics.asterix.om.types.AOrderedListType;
-import edu.uci.ics.asterix.om.types.BuiltinType;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
-
-/**
- * Provides functionality for writing parameters for a FEED dataset into the
- * Metadata. Since FEED dataset is a special kind of INTERNAL dataset, this
- * class extends InternalDatasetDetails.
- */
-public class FeedDatasetDetails extends InternalDatasetDetails {
-
- private static final long serialVersionUID = 1L;
- private final String adapterFactory;
- private final Map<String, String> properties;
- private final FunctionSignature signature;
- private FeedState feedState;
-
- public enum FeedState {
- INACTIVE,
- // INACTIVE state signifies that the feed dataset is not
- // connected with the external world through the feed
- // adapter.
- ACTIVE
- // ACTIVE state signifies that the feed dataset is connected to the
- // external world using an adapter that may put data into the dataset.
- }
-
- public FeedDatasetDetails(FileStructure fileStructure, PartitioningStrategy partitioningStrategy,
- List<String> partitioningKey, List<String> primaryKey, String groupName, String adapterFactory,
- Map<String, String> properties, FunctionSignature signature, String feedState, String compactionPolicy,
- Map<String, String> compactionPolicyProperties) {
- super(fileStructure, partitioningStrategy, partitioningKey, primaryKey, groupName, compactionPolicy,
- compactionPolicyProperties);
- this.properties = properties;
- this.adapterFactory = adapterFactory;
- this.signature = signature;
- this.feedState = feedState.equals(FeedState.ACTIVE.toString()) ? FeedState.ACTIVE : FeedState.INACTIVE;
- }
-
- @Override
- public DatasetType getDatasetType() {
- return DatasetType.FEED;
- }
-
- @Override
- public void writeDatasetDetailsRecordType(DataOutput out) throws HyracksDataException {
- IARecordBuilder feedRecordBuilder = new RecordBuilder();
- OrderedListBuilder listBuilder = new OrderedListBuilder();
- ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
- ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
- feedRecordBuilder.reset(MetadataRecordTypes.FEED_DETAILS_RECORDTYPE);
- AMutableString aString = new AMutableString("");
- ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ASTRING);
-
- // write field 0
- fieldValue.reset();
- aString.setValue(this.getFileStructure().toString());
- stringSerde.serialize(aString, fieldValue.getDataOutput());
- feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_FILESTRUCTURE_FIELD_INDEX, fieldValue);
-
- // write field 1
- fieldValue.reset();
- aString.setValue(this.getPartitioningStrategy().toString());
- stringSerde.serialize(aString, fieldValue.getDataOutput());
- feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_PARTITIONSTRATEGY_FIELD_INDEX, fieldValue);
-
- // write field 2
- listBuilder.reset((AOrderedListType) MetadataRecordTypes.FEED_DETAILS_RECORDTYPE.getFieldTypes()[2]);
- for (String field : partitioningKeys) {
- itemValue.reset();
- aString.setValue(field);
- stringSerde.serialize(aString, itemValue.getDataOutput());
- listBuilder.addItem(itemValue);
- }
- fieldValue.reset();
- listBuilder.write(fieldValue.getDataOutput(), true);
- feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_PARTITIONKEY_FIELD_INDEX, fieldValue);
-
- // write field 3
- listBuilder.reset((AOrderedListType) MetadataRecordTypes.FEED_DETAILS_RECORDTYPE.getFieldTypes()[3]);
- for (String field : primaryKeys) {
- itemValue.reset();
- aString.setValue(field);
- stringSerde.serialize(aString, itemValue.getDataOutput());
- listBuilder.addItem(itemValue);
- }
- fieldValue.reset();
- listBuilder.write(fieldValue.getDataOutput(), true);
- feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_PRIMARYKEY_FIELD_INDEX, fieldValue);
-
- // write field 4
- fieldValue.reset();
- aString.setValue(getNodeGroupName());
- stringSerde.serialize(aString, fieldValue.getDataOutput());
- feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_GROUPNAME_FIELD_INDEX, fieldValue);
-
- // write field 5
- fieldValue.reset();
- aString.setValue(getAdapterFactory());
- stringSerde.serialize(aString, fieldValue.getDataOutput());
- feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_DATASOURCE_ADAPTER_FIELD_INDEX, fieldValue);
-
- // write field 6
- listBuilder.reset((AOrderedListType) MetadataRecordTypes.FEED_DETAILS_RECORDTYPE.getFieldTypes()[6]);
- for (Map.Entry<String, String> property : properties.entrySet()) {
- String name = property.getKey();
- String value = property.getValue();
- itemValue.reset();
- writePropertyTypeRecord(name, value, itemValue.getDataOutput(),
- MetadataRecordTypes.DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE);
- listBuilder.addItem(itemValue);
- }
- fieldValue.reset();
- listBuilder.write(fieldValue.getDataOutput(), true);
- feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_PROPERTIES_FIELD_INDEX, fieldValue);
-
- // write field 7
- fieldValue.reset();
- if (signature != null) {
- aString.setValue(signature.toString());
- stringSerde.serialize(aString, fieldValue.getDataOutput());
- feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_FUNCTION_FIELD_INDEX, fieldValue);
- }
-
- // write field 8
- fieldValue.reset();
- aString.setValue(getFeedState().toString());
- stringSerde.serialize(aString, fieldValue.getDataOutput());
- feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_STATE_FIELD_INDEX, fieldValue);
-
- // write field 9
- fieldValue.reset();
- aString.setValue(getCompactionPolicy().toString());
- stringSerde.serialize(aString, fieldValue.getDataOutput());
- feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_COMPACTION_POLICY_FIELD_INDEX, fieldValue);
-
- // write field 10
- listBuilder.reset((AOrderedListType) MetadataRecordTypes.FEED_DETAILS_RECORDTYPE.getFieldTypes()[10]);
- for (Map.Entry<String, String> property : compactionPolicyProperties.entrySet()) {
- String name = property.getKey();
- String value = property.getValue();
- itemValue.reset();
- writePropertyTypeRecord(name, value, itemValue.getDataOutput(),
- MetadataRecordTypes.COMPACTION_POLICY_PROPERTIES_RECORDTYPE);
- listBuilder.addItem(itemValue);
- }
- fieldValue.reset();
- listBuilder.write(fieldValue.getDataOutput(), true);
- feedRecordBuilder.addField(MetadataRecordTypes.FEED_DETAILS_ARECORD_COMPACTION_POLICY_PROPERTIES_FIELD_INDEX,
- fieldValue);
-
- try {
- feedRecordBuilder.write(out, true);
- } catch (IOException | AsterixException e) {
- throw new HyracksDataException(e);
- }
-
- }
-
- public FeedState getFeedState() {
- return feedState;
- }
-
- public void setFeedState(FeedState feedState) {
- this.feedState = feedState;
- }
-
- public String getAdapterFactory() {
- return adapterFactory;
- }
-
- public Map<String, String> getProperties() {
- return properties;
- }
-
- public FunctionSignature getFunction() {
- return signature;
- }
-
-}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedPolicy.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedPolicy.java
new file mode 100644
index 0000000..b011e5c
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedPolicy.java
@@ -0,0 +1,95 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.metadata.entities;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.metadata.MetadataCache;
+import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
+
+/**
+ * Metadata describing a feed activity record.
+ */
+public class FeedPolicy implements IMetadataEntity {
+
+ private static final long serialVersionUID = 1L;
+
+ private final String dataverseName;
+ // Enforced to be unique within a dataverse.
+ private final String policyName;
+ // A description of the policy
+ private final String description;
+ // The policy properties associated with the feed dataset
+ private Map<String, String> properties;
+
+ public FeedPolicy(String dataverseName, String policyName, String description, Map<String, String> properties) {
+ this.dataverseName = dataverseName;
+ this.policyName = policyName;
+ this.description = description;
+ this.properties = properties;
+ }
+
+ public String getDataverseName() {
+ return dataverseName;
+ }
+
+ public String getPolicyName() {
+ return policyName;
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (this == other) {
+ return true;
+ }
+ if (!(other instanceof FeedPolicy)) {
+ return false;
+ }
+ FeedPolicy otherPolicy = (FeedPolicy) other;
+ if (!otherPolicy.dataverseName.equals(dataverseName)) {
+ return false;
+ }
+ if (!otherPolicy.policyName.equals(policyName)) {
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ public Object addToCache(MetadataCache cache) {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Object dropFromCache(MetadataCache cache) {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ public String getDescription() {
+ return description;
+ }
+
+ public Map<String, String> getProperties() {
+ return properties;
+ }
+
+ public void setProperties(Map<String, String> properties) {
+ this.properties = properties;
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Library.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Library.java
new file mode 100644
index 0000000..a341ba9
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Library.java
@@ -0,0 +1,36 @@
+package edu.uci.ics.asterix.metadata.entities;
+
+import edu.uci.ics.asterix.metadata.MetadataCache;
+import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
+
+public class Library implements IMetadataEntity {
+
+ private static final long serialVersionUID = 1L;
+
+ private final String dataverse;
+ private final String name;
+
+ public Library(String dataverseName, String libraryName) {
+ this.dataverse = dataverseName;
+ this.name = libraryName;
+ }
+
+ public String getDataverseName() {
+ return dataverse;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public Object addToCache(MetadataCache cache) {
+ return cache.addLibraryIfNotExists(this);
+ }
+
+ @Override
+ public Object dropFromCache(MetadataCache cache) {
+ return cache.dropLibrary(this);
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java
index dc8c390..a85787f 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java
@@ -20,6 +20,7 @@
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import edu.uci.ics.asterix.metadata.api.IMetadataEntityTupleTranslator;
import edu.uci.ics.asterix.om.base.ABoolean;
+import edu.uci.ics.asterix.om.base.AInt32;
import edu.uci.ics.asterix.om.base.AMutableString;
import edu.uci.ics.asterix.om.base.AString;
import edu.uci.ics.asterix.om.types.BuiltinType;
@@ -40,6 +41,10 @@
@SuppressWarnings("unchecked")
protected ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.ABOOLEAN);
+ @SuppressWarnings("unchecked")
+ protected ISerializerDeserializer<AInt32> int32Serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT32);
+
protected final IARecordBuilder recordBuilder;
protected final ArrayBackedValueStorage fieldValue;
protected final ArrayTupleBuilder tupleBuilder;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index 3f70e2b..2bf666c 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -31,7 +31,6 @@
import edu.uci.ics.asterix.builders.UnorderedListBuilder;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.functions.FunctionSignature;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import edu.uci.ics.asterix.metadata.IDatasetDetails;
import edu.uci.ics.asterix.metadata.MetadataException;
@@ -39,14 +38,12 @@
import edu.uci.ics.asterix.metadata.bootstrap.MetadataRecordTypes;
import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
-import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails.FileStructure;
import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
import edu.uci.ics.asterix.om.base.AInt32;
import edu.uci.ics.asterix.om.base.AMutableInt32;
import edu.uci.ics.asterix.om.base.AMutableString;
-import edu.uci.ics.asterix.om.base.ANull;
import edu.uci.ics.asterix.om.base.AOrderedList;
import edu.uci.ics.asterix.om.base.ARecord;
import edu.uci.ics.asterix.om.base.AString;
@@ -110,86 +107,7 @@
int pendingOp = ((AInt32) datasetRecord
.getValueByPos(MetadataRecordTypes.DATASET_ARECORD_PENDINGOP_FIELD_INDEX)).getIntegerValue();
switch (datasetType) {
- case FEED: {
- ARecord datasetDetailsRecord = (ARecord) datasetRecord
- .getValueByPos(MetadataRecordTypes.DATASET_ARECORD_FEEDDETAILS_FIELD_INDEX);
- FileStructure fileStructure = FileStructure.valueOf(((AString) datasetDetailsRecord
- .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_FILESTRUCTURE_FIELD_INDEX))
- .getStringValue());
- PartitioningStrategy partitioningStrategy = PartitioningStrategy
- .valueOf(((AString) datasetDetailsRecord
- .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_PARTITIONSTRATEGY_FIELD_INDEX))
- .getStringValue());
- IACursor cursor = ((AOrderedList) datasetDetailsRecord
- .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_PARTITIONKEY_FIELD_INDEX)).getCursor();
- List<String> partitioningKey = new ArrayList<String>();
- while (cursor.next())
- partitioningKey.add(((AString) cursor.get()).getStringValue());
- String groupName = ((AString) datasetDetailsRecord
- .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_GROUPNAME_FIELD_INDEX))
- .getStringValue();
- String adapter = ((AString) datasetDetailsRecord
- .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_DATASOURCE_ADAPTER_FIELD_INDEX))
- .getStringValue();
- cursor = ((AOrderedList) datasetDetailsRecord
- .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_PROPERTIES_FIELD_INDEX)).getCursor();
- Map<String, String> properties = new HashMap<String, String>();
- String key;
- String value;
- while (cursor.next()) {
- ARecord field = (ARecord) cursor.get();
- key = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_NAME_FIELD_INDEX))
- .getStringValue();
- value = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_VALUE_FIELD_INDEX))
- .getStringValue();
- properties.put(key, value);
- }
- Object o = datasetDetailsRecord
- .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_FUNCTION_FIELD_INDEX);
- FunctionSignature signature = null;
- if (!(o instanceof ANull)) {
- String functionIdentifier = ((AString) o).getStringValue();
- String[] qnameComponents = functionIdentifier.split("\\.");
- String functionDataverse;
- String functionName;
- if (qnameComponents.length == 2) {
- functionDataverse = qnameComponents[0];
- functionName = qnameComponents[1];
- } else {
- functionDataverse = dataverseName;
- functionName = qnameComponents[0];
- }
-
- String[] nameComponents = functionName.split("@");
- signature = new FunctionSignature(functionDataverse, nameComponents[0],
- Integer.parseInt(nameComponents[1]));
- }
-
- String feedState = ((AString) datasetDetailsRecord
- .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_STATE_FIELD_INDEX)).getStringValue();
-
- String compactionPolicy = ((AString) datasetDetailsRecord
- .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_COMPACTION_POLICY_FIELD_INDEX))
- .getStringValue();
- cursor = ((AOrderedList) datasetDetailsRecord
- .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_COMPACTION_POLICY_PROPERTIES_FIELD_INDEX))
- .getCursor();
- Map<String, String> compactionPolicyProperties = new HashMap<String, String>();
- while (cursor.next()) {
- ARecord field = (ARecord) cursor.get();
- key = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_NAME_FIELD_INDEX))
- .getStringValue();
- value = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_VALUE_FIELD_INDEX))
- .getStringValue();
- compactionPolicyProperties.put(key, value);
- }
-
- datasetDetails = new FeedDatasetDetails(fileStructure, partitioningStrategy, partitioningKey,
- partitioningKey, groupName, adapter, properties, signature, feedState, compactionPolicy,
- compactionPolicyProperties);
- break;
- }
case INTERNAL: {
ARecord datasetDetailsRecord = (ARecord) datasetRecord
.getValueByPos(MetadataRecordTypes.DATASET_ARECORD_INTERNALDETAILS_FIELD_INDEX);
@@ -363,9 +281,6 @@
case EXTERNAL:
recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_EXTERNALDETAILS_FIELD_INDEX, fieldValue);
break;
- case FEED:
- recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_FEEDDETAILS_FIELD_INDEX, fieldValue);
- break;
}
}
@@ -379,10 +294,8 @@
IACursor cursor = list.getCursor();
while (cursor.next()) {
ARecord field = (ARecord) cursor.get();
- key = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_NAME_FIELD_INDEX))
- .getStringValue();
- value = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_VALUE_FIELD_INDEX))
- .getStringValue();
+ key = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_NAME_FIELD_INDEX)).getStringValue();
+ value = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_VALUE_FIELD_INDEX)).getStringValue();
hints.put(key, value);
}
return hints;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
index 4a5e4dcf..e93fd97 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
@@ -22,13 +22,13 @@
import java.util.Calendar;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.external.dataset.adapter.AdapterIdentifier;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
import edu.uci.ics.asterix.metadata.bootstrap.MetadataRecordTypes;
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter.AdapterType;
+import edu.uci.ics.asterix.metadata.feeds.AdapterIdentifier;
import edu.uci.ics.asterix.om.base.ARecord;
import edu.uci.ics.asterix.om.base.AString;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedActivityTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedActivityTupleTranslator.java
new file mode 100644
index 0000000..bcc16a7
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedActivityTupleTranslator.java
@@ -0,0 +1,244 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.metadata.entitytupletranslators;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.builders.IARecordBuilder;
+import edu.uci.ics.asterix.builders.RecordBuilder;
+import edu.uci.ics.asterix.builders.UnorderedListBuilder;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataRecordTypes;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.ARecord;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.base.AUnorderedList;
+import edu.uci.ics.asterix.om.base.IACursor;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * Translates a Dataset metadata entity to an ITupleReference and vice versa.
+ */
+public class FeedActivityTupleTranslator extends AbstractTupleTranslator<FeedActivity> {
+ // Field indexes of serialized FeedActivity in a tuple.
+ // Key field.
+ public static final int FEED_ACTIVITY_ACTIVITY_DATAVERSE_NAME_FIELD_INDEX = 0;
+
+ public static final int FEED_ACTIVITY_ACTIVITY_FEED_NAME_FIELD_INDEX = 1;
+
+ public static final int FEED_ACTIVITY_ACTIVITY_DATASET_NAME_FIELD_INDEX = 2;
+
+ public static final int FEED_ACTIVITY_ACTIVITY_ID_FIELD_INDEX = 3;
+
+ // Payload field containing serialized FeedActivity.
+ public static final int FEED_ACTIVITY_PAYLOAD_TUPLE_FIELD_INDEX = 4;
+
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(MetadataRecordTypes.FEED_ACTIVITY_RECORDTYPE);
+ private AMutableInt32 aInt32;
+ protected ISerializerDeserializer<AInt32> aInt32Serde;
+
+ @SuppressWarnings("unchecked")
+ public FeedActivityTupleTranslator(boolean getTuple) {
+ super(getTuple, MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET.getFieldCount());
+ aInt32 = new AMutableInt32(-1);
+ aInt32Serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+ }
+
+ @Override
+ public FeedActivity getMetadataEntytiFromTuple(ITupleReference frameTuple) throws IOException {
+ byte[] serRecord = frameTuple.getFieldData(FEED_ACTIVITY_PAYLOAD_TUPLE_FIELD_INDEX);
+ int recordStartOffset = frameTuple.getFieldStart(FEED_ACTIVITY_PAYLOAD_TUPLE_FIELD_INDEX);
+ int recordLength = frameTuple.getFieldLength(FEED_ACTIVITY_PAYLOAD_TUPLE_FIELD_INDEX);
+ ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
+ DataInput in = new DataInputStream(stream);
+ ARecord feedActivityRecord = (ARecord) recordSerDes.deserialize(in);
+ return createFeedActivityFromARecord(feedActivityRecord);
+ }
+
+ private FeedActivity createFeedActivityFromARecord(ARecord feedActivityRecord) {
+
+ String dataverseName = ((AString) feedActivityRecord
+ .getValueByPos(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_DATAVERSE_NAME_FIELD_INDEX)).getStringValue();
+ String feedName = ((AString) feedActivityRecord
+ .getValueByPos(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_FEED_NAME_FIELD_INDEX)).getStringValue();
+ String datasetName = ((AString) feedActivityRecord
+ .getValueByPos(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_DATASET_NAME_FIELD_INDEX)).getStringValue();
+ int activityId = ((AInt32) feedActivityRecord
+ .getValueByPos(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_ACTIVITY_ID_FIELD_INDEX)).getIntegerValue();
+ String feedActivityType = ((AString) feedActivityRecord
+ .getValueByPos(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_ACTIVITY_TYPE_FIELD_INDEX)).getStringValue();
+
+ IACursor cursor = ((AUnorderedList) feedActivityRecord
+ .getValueByPos(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_DETAILS_FIELD_INDEX)).getCursor();
+ Map<String, String> activityDetails = new HashMap<String, String>();
+ String key;
+ String value;
+ while (cursor.next()) {
+ ARecord field = (ARecord) cursor.get();
+ key = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_NAME_FIELD_INDEX)).getStringValue();
+ value = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_VALUE_FIELD_INDEX)).getStringValue();
+ activityDetails.put(key, value);
+ }
+
+ String feedActivityTimestamp = ((AString) feedActivityRecord
+ .getValueByPos(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_LAST_UPDATE_TIMESTAMP_FIELD_INDEX))
+ .getStringValue();
+
+ FeedActivity fa = new FeedActivity(dataverseName, feedName, datasetName,
+ FeedActivityType.valueOf(feedActivityType), activityDetails);
+ fa.setLastUpdatedTimestamp(feedActivityTimestamp);
+ fa.setActivityId(activityId);
+ return fa;
+ }
+
+ @Override
+ public ITupleReference getTupleFromMetadataEntity(FeedActivity feedActivity) throws IOException, MetadataException {
+ // write the key in the first three fields of the tuple
+ ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
+
+ tupleBuilder.reset();
+ aString.setValue(feedActivity.getDataverseName());
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+
+ aString.setValue(feedActivity.getFeedName());
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+
+ aString.setValue(feedActivity.getDatasetName());
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+
+ aInt32.setValue(feedActivity.getActivityId());
+ int32Serde.serialize(aInt32, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+ // write the pay-load in the 2nd field of the tuple
+
+ recordBuilder.reset(MetadataRecordTypes.FEED_ACTIVITY_RECORDTYPE);
+
+ // write field 0
+ fieldValue.reset();
+ aString.setValue(feedActivity.getDataverseName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_DATAVERSE_NAME_FIELD_INDEX, fieldValue);
+
+ // write field 1
+ fieldValue.reset();
+ aString.setValue(feedActivity.getFeedName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_FEED_NAME_FIELD_INDEX, fieldValue);
+
+ // write field 2
+ fieldValue.reset();
+ aString.setValue(feedActivity.getDatasetName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_DATASET_NAME_FIELD_INDEX, fieldValue);
+
+ // write field 3
+ fieldValue.reset();
+ aInt32.setValue(feedActivity.getActivityId());
+ int32Serde.serialize(aInt32, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_ACTIVITY_ID_FIELD_INDEX, fieldValue);
+
+ // write field 4
+ fieldValue.reset();
+ aString.setValue(feedActivity.getFeedActivityType().name());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_ACTIVITY_TYPE_FIELD_INDEX, fieldValue);
+
+ // write field 5
+ Map<String, String> properties = feedActivity.getFeedActivityDetails();
+ UnorderedListBuilder listBuilder = new UnorderedListBuilder();
+ listBuilder
+ .reset((AUnorderedListType) MetadataRecordTypes.FEED_ACTIVITY_RECORDTYPE.getFieldTypes()[MetadataRecordTypes.FEED_ACTIVITY_ARECORD_DETAILS_FIELD_INDEX]);
+ for (Map.Entry<String, String> property : properties.entrySet()) {
+ String name = property.getKey();
+ String value = property.getValue();
+ itemValue.reset();
+ writePropertyTypeRecord(name, value, itemValue.getDataOutput());
+ listBuilder.addItem(itemValue);
+ }
+ fieldValue.reset();
+ listBuilder.write(fieldValue.getDataOutput(), true);
+ recordBuilder.addField(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_DETAILS_FIELD_INDEX, fieldValue);
+
+ // write field 6
+ fieldValue.reset();
+ aString.setValue(Calendar.getInstance().getTime().toString());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.FEED_ACTIVITY_ARECORD_LAST_UPDATE_TIMESTAMP_FIELD_INDEX, fieldValue);
+
+ // write record
+ try {
+ recordBuilder.write(tupleBuilder.getDataOutput(), true);
+ } catch (AsterixException e) {
+ throw new MetadataException(e);
+ }
+ tupleBuilder.addFieldEndOffset();
+
+ tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+ return tuple;
+ }
+
+ public void writePropertyTypeRecord(String name, String value, DataOutput out) throws HyracksDataException {
+ IARecordBuilder propertyRecordBuilder = new RecordBuilder();
+ ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
+ propertyRecordBuilder.reset(MetadataRecordTypes.FEED_ACTIVITY_DETAILS_RECORDTYPE);
+ AMutableString aString = new AMutableString("");
+ ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ASTRING);
+
+ // write field 0
+ fieldValue.reset();
+ aString.setValue(name);
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ propertyRecordBuilder.addField(0, fieldValue);
+
+ // write field 1
+ fieldValue.reset();
+ aString.setValue(value);
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ propertyRecordBuilder.addField(1, fieldValue);
+
+ try {
+ propertyRecordBuilder.write(out, true);
+ } catch (IOException | AsterixException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java
new file mode 100644
index 0000000..57d2d1c
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java
@@ -0,0 +1,200 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.metadata.entitytupletranslators;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.builders.IARecordBuilder;
+import edu.uci.ics.asterix.builders.RecordBuilder;
+import edu.uci.ics.asterix.builders.UnorderedListBuilder;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataRecordTypes;
+import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.ARecord;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.base.AUnorderedList;
+import edu.uci.ics.asterix.om.base.IACursor;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * Translates a Dataset metadata entity to an ITupleReference and vice versa.
+ */
+public class FeedPolicyTupleTranslator extends AbstractTupleTranslator<FeedPolicy> {
+ // Field indexes of serialized FeedPolicy in a tuple.
+ // Key field.
+ public static final int FEED_POLICY_DATAVERSE_NAME_FIELD_INDEX = 0;
+
+ public static final int FEED_POLICY_POLICY_NAME_FIELD_INDEX = 1;
+
+ // Payload field containing serialized feedPolicy.
+ public static final int FEED_POLICY_PAYLOAD_TUPLE_FIELD_INDEX = 2;
+
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(MetadataRecordTypes.FEED_POLICY_RECORDTYPE);
+ private AMutableInt32 aInt32;
+ protected ISerializerDeserializer<AInt32> aInt32Serde;
+
+ @SuppressWarnings("unchecked")
+ public FeedPolicyTupleTranslator(boolean getTuple) {
+ super(getTuple, MetadataPrimaryIndexes.FEED_POLICY_DATASET.getFieldCount());
+ aInt32 = new AMutableInt32(-1);
+ aInt32Serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+ }
+
+ @Override
+ public FeedPolicy getMetadataEntytiFromTuple(ITupleReference frameTuple) throws IOException {
+ byte[] serRecord = frameTuple.getFieldData(FEED_POLICY_PAYLOAD_TUPLE_FIELD_INDEX);
+ int recordStartOffset = frameTuple.getFieldStart(FEED_POLICY_PAYLOAD_TUPLE_FIELD_INDEX);
+ int recordLength = frameTuple.getFieldLength(FEED_POLICY_PAYLOAD_TUPLE_FIELD_INDEX);
+ ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
+ DataInput in = new DataInputStream(stream);
+ ARecord feedPolicyRecord = (ARecord) recordSerDes.deserialize(in);
+ return createFeedPolicyFromARecord(feedPolicyRecord);
+ }
+
+ private FeedPolicy createFeedPolicyFromARecord(ARecord feedPolicyRecord) {
+ FeedPolicy feedPolicy = null;
+ String dataverseName = ((AString) feedPolicyRecord
+ .getValueByPos(MetadataRecordTypes.FEED_POLICY_ARECORD_DATAVERSE_NAME_FIELD_INDEX)).getStringValue();
+ String policyName = ((AString) feedPolicyRecord
+ .getValueByPos(MetadataRecordTypes.FEED_POLICY_ARECORD_POLICY_NAME_FIELD_INDEX)).getStringValue();
+
+ String description = ((AString) feedPolicyRecord
+ .getValueByPos(MetadataRecordTypes.FEED_POLICY_ARECORD_DESCRIPTION_FIELD_INDEX)).getStringValue();
+
+ IACursor cursor = ((AUnorderedList) feedPolicyRecord
+ .getValueByPos(MetadataRecordTypes.FEED_POLICY_ARECORD_PROPERTIES_FIELD_INDEX)).getCursor();
+ Map<String, String> policyParamters = new HashMap<String, String>();
+ String key;
+ String value;
+ while (cursor.next()) {
+ ARecord field = (ARecord) cursor.get();
+ key = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_NAME_FIELD_INDEX)).getStringValue();
+ value = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_VALUE_FIELD_INDEX)).getStringValue();
+ policyParamters.put(key, value);
+ }
+
+ feedPolicy = new FeedPolicy(dataverseName, policyName, description, policyParamters);
+ return feedPolicy;
+ }
+
+ @Override
+ public ITupleReference getTupleFromMetadataEntity(FeedPolicy feedPolicy) throws IOException, MetadataException {
+ // write the key in the first three fields of the tuple
+ ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
+
+ tupleBuilder.reset();
+ aString.setValue(feedPolicy.getDataverseName());
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+
+ aString.setValue(feedPolicy.getPolicyName());
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+
+ recordBuilder.reset(MetadataRecordTypes.FEED_POLICY_RECORDTYPE);
+
+ // write field 0
+ fieldValue.reset();
+ aString.setValue(feedPolicy.getDataverseName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.FEED_POLICY_ARECORD_DATAVERSE_NAME_FIELD_INDEX, fieldValue);
+
+ // write field 1
+ fieldValue.reset();
+ aString.setValue(feedPolicy.getPolicyName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.FEED_POLICY_ARECORD_POLICY_NAME_FIELD_INDEX, fieldValue);
+
+ // write field 2
+ fieldValue.reset();
+ aString.setValue(feedPolicy.getDescription());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.FEED_POLICY_ARECORD_POLICY_NAME_FIELD_INDEX, fieldValue);
+
+ // write field 3 (properties)
+ Map<String, String> properties = feedPolicy.getProperties();
+ UnorderedListBuilder listBuilder = new UnorderedListBuilder();
+ listBuilder
+ .reset((AUnorderedListType) MetadataRecordTypes.FEED_POLICY_RECORDTYPE.getFieldTypes()[MetadataRecordTypes.FEED_POLICY_ARECORD_PROPERTIES_FIELD_INDEX]);
+ for (Map.Entry<String, String> property : properties.entrySet()) {
+ String name = property.getKey();
+ String value = property.getValue();
+ itemValue.reset();
+ writePropertyTypeRecord(name, value, itemValue.getDataOutput());
+ listBuilder.addItem(itemValue);
+ }
+ fieldValue.reset();
+ listBuilder.write(fieldValue.getDataOutput(), true);
+ recordBuilder.addField(MetadataRecordTypes.FEED_POLICY_ARECORD_PROPERTIES_FIELD_INDEX, fieldValue);
+
+ // write record
+ try {
+ recordBuilder.write(tupleBuilder.getDataOutput(), true);
+ } catch (AsterixException e) {
+ throw new MetadataException(e);
+ }
+ tupleBuilder.addFieldEndOffset();
+
+ tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+ return tuple;
+ }
+
+ public void writePropertyTypeRecord(String name, String value, DataOutput out) throws HyracksDataException {
+ IARecordBuilder propertyRecordBuilder = new RecordBuilder();
+ ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
+ propertyRecordBuilder.reset(MetadataRecordTypes.POLICY_PARAMS_RECORDTYPE);
+ AMutableString aString = new AMutableString("");
+ ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ASTRING);
+
+ // write field 0
+ fieldValue.reset();
+ aString.setValue(name);
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ propertyRecordBuilder.addField(0, fieldValue);
+
+ // write field 1
+ fieldValue.reset();
+ aString.setValue(value);
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ propertyRecordBuilder.addField(1, fieldValue);
+
+ try {
+ propertyRecordBuilder.write(out, true);
+ } catch (IOException | AsterixException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java
new file mode 100644
index 0000000..7de0118
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java
@@ -0,0 +1,235 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.metadata.entitytupletranslators;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.builders.IARecordBuilder;
+import edu.uci.ics.asterix.builders.RecordBuilder;
+import edu.uci.ics.asterix.builders.UnorderedListBuilder;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataRecordTypes;
+import edu.uci.ics.asterix.metadata.entities.Feed;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.ARecord;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.base.AUnorderedList;
+import edu.uci.ics.asterix.om.base.IACursor;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * Translates a Feed metadata entity to an ITupleReference and vice versa.
+ */
+public class FeedTupleTranslator extends AbstractTupleTranslator<Feed> {
+ // Field indexes of serialized Feed in a tuple.
+ // Key field.
+ public static final int FEED_DATAVERSE_NAME_FIELD_INDEX = 0;
+
+ public static final int FEED_NAME_FIELD_INDEX = 1;
+
+ // Payload field containing serialized feed.
+ public static final int FEED_PAYLOAD_TUPLE_FIELD_INDEX = 2;
+
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(MetadataRecordTypes.FEED_RECORDTYPE);
+ private AMutableInt32 aInt32;
+ protected ISerializerDeserializer<AInt32> aInt32Serde;
+
+ @SuppressWarnings("unchecked")
+ public FeedTupleTranslator(boolean getTuple) {
+ super(getTuple, MetadataPrimaryIndexes.FEED_DATASET.getFieldCount());
+ aInt32 = new AMutableInt32(-1);
+ aInt32Serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+ }
+
+ @Override
+ public Feed getMetadataEntytiFromTuple(ITupleReference frameTuple) throws IOException {
+ byte[] serRecord = frameTuple.getFieldData(FEED_PAYLOAD_TUPLE_FIELD_INDEX);
+ int recordStartOffset = frameTuple.getFieldStart(FEED_PAYLOAD_TUPLE_FIELD_INDEX);
+ int recordLength = frameTuple.getFieldLength(FEED_PAYLOAD_TUPLE_FIELD_INDEX);
+ ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
+ DataInput in = new DataInputStream(stream);
+ ARecord feedRecord = (ARecord) recordSerDes.deserialize(in);
+ return createFeedFromARecord(feedRecord);
+ }
+
+ private Feed createFeedFromARecord(ARecord feedRecord) {
+ Feed feed = null;
+ String dataverseName = ((AString) feedRecord
+ .getValueByPos(MetadataRecordTypes.FEED_ARECORD_DATAVERSE_NAME_FIELD_INDEX)).getStringValue();
+ String feedName = ((AString) feedRecord.getValueByPos(MetadataRecordTypes.FEED_ARECORD_FEED_NAME_FIELD_INDEX))
+ .getStringValue();
+ String adaptorName = ((AString) feedRecord
+ .getValueByPos(MetadataRecordTypes.FEED_ARECORD_ADAPTOR_NAME_FIELD_INDEX)).getStringValue();
+
+ IACursor cursor = ((AUnorderedList) feedRecord
+ .getValueByPos(MetadataRecordTypes.FEED_ARECORD_ADAPTOR_CONFIGURATION_FIELD_INDEX)).getCursor();
+ String key;
+ String value;
+ Map<String, String> adaptorConfiguration = new HashMap<String, String>();
+ while (cursor.next()) {
+ ARecord field = (ARecord) cursor.get();
+ key = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_NAME_FIELD_INDEX)).getStringValue();
+ value = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_VALUE_FIELD_INDEX)).getStringValue();
+ adaptorConfiguration.put(key, value);
+ }
+
+ Object o = feedRecord.getValueByPos(MetadataRecordTypes.FEED_ARECORD_FUNCTION_FIELD_INDEX);
+ FunctionSignature signature = null;
+ if (!(o instanceof ANull)) {
+ String functionIdentifier = ((AString) o).getStringValue();
+ String[] qnameComponents = functionIdentifier.split("\\.");
+ String functionDataverse;
+ String functionName;
+ if (qnameComponents.length == 2) {
+ functionDataverse = qnameComponents[0];
+ functionName = qnameComponents[1];
+ } else {
+ functionDataverse = dataverseName;
+ functionName = qnameComponents[0];
+ }
+
+ String[] nameComponents = functionName.split("@");
+ signature = new FunctionSignature(functionDataverse, nameComponents[0], Integer.parseInt(nameComponents[1]));
+ }
+
+ feed = new Feed(dataverseName, feedName, adaptorName, adaptorConfiguration, signature);
+ return feed;
+ }
+
+ @Override
+ public ITupleReference getTupleFromMetadataEntity(Feed feed) throws IOException, MetadataException {
+ // write the key in the first three fields of the tuple
+ ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
+
+ tupleBuilder.reset();
+ aString.setValue(feed.getDataverseName());
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+
+ aString.setValue(feed.getFeedName());
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+
+ recordBuilder.reset(MetadataRecordTypes.FEED_RECORDTYPE);
+
+ // write field 0
+ fieldValue.reset();
+ aString.setValue(feed.getDataverseName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_DATAVERSE_NAME_FIELD_INDEX, fieldValue);
+
+ // write field 1
+ fieldValue.reset();
+ aString.setValue(feed.getFeedName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_FEED_NAME_FIELD_INDEX, fieldValue);
+
+ // write field 2
+ fieldValue.reset();
+ aString.setValue(feed.getAdaptorName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_ADAPTOR_NAME_FIELD_INDEX, fieldValue);
+
+ // write field 3 (adaptorConfiguration)
+ Map<String, String> adaptorConfiguration = feed.getAdaptorConfiguration();
+ UnorderedListBuilder listBuilder = new UnorderedListBuilder();
+ listBuilder
+ .reset((AUnorderedListType) MetadataRecordTypes.FEED_RECORDTYPE.getFieldTypes()[MetadataRecordTypes.FEED_ARECORD_ADAPTOR_CONFIGURATION_FIELD_INDEX]);
+ for (Map.Entry<String, String> property : adaptorConfiguration.entrySet()) {
+ String name = property.getKey();
+ String value = property.getValue();
+ itemValue.reset();
+ writePropertyTypeRecord(name, value, itemValue.getDataOutput());
+ listBuilder.addItem(itemValue);
+ }
+ fieldValue.reset();
+ listBuilder.write(fieldValue.getDataOutput(), true);
+ recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_ADAPTOR_CONFIGURATION_FIELD_INDEX, fieldValue);
+
+ // write field 4
+ fieldValue.reset();
+ if (feed.getAppliedFunction() != null) {
+ aString.setValue(feed.getAppliedFunction().toString());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_FUNCTION_FIELD_INDEX, fieldValue);
+ }
+
+ // write field 5
+ fieldValue.reset();
+ aString.setValue(Calendar.getInstance().getTime().toString());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_TIMESTAMP_FIELD_INDEX, fieldValue);
+
+ // write record
+ try {
+ recordBuilder.write(tupleBuilder.getDataOutput(), true);
+ } catch (AsterixException e) {
+ throw new MetadataException(e);
+ }
+ tupleBuilder.addFieldEndOffset();
+
+ tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+ return tuple;
+ }
+
+ public void writePropertyTypeRecord(String name, String value, DataOutput out) throws HyracksDataException {
+ IARecordBuilder propertyRecordBuilder = new RecordBuilder();
+ ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
+ propertyRecordBuilder.reset(MetadataRecordTypes.FEED_ADAPTOR_CONFIGURATION_RECORDTYPE);
+ AMutableString aString = new AMutableString("");
+ ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ASTRING);
+
+ // write field 0
+ fieldValue.reset();
+ aString.setValue(name);
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ propertyRecordBuilder.addField(0, fieldValue);
+
+ // write field 1
+ fieldValue.reset();
+ aString.setValue(value);
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ propertyRecordBuilder.addField(1, fieldValue);
+
+ try {
+ propertyRecordBuilder.write(out, true);
+ } catch (IOException | AsterixException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
new file mode 100644
index 0000000..4bd2366
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
@@ -0,0 +1,120 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.metadata.entitytupletranslators;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.Calendar;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataRecordTypes;
+import edu.uci.ics.asterix.metadata.entities.Library;
+import edu.uci.ics.asterix.om.base.ARecord;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * Translates a Library metadata entity to an ITupleReference and vice versa.
+ */
+public class LibraryTupleTranslator extends AbstractTupleTranslator<Library> {
+ // Field indexes of serialized Library in a tuple.
+ // First key field.
+ public static final int LIBRARY_DATAVERSENAME_TUPLE_FIELD_INDEX = 0;
+ // Second key field.
+ public static final int LIBRARY_NAME_TUPLE_FIELD_INDEX = 1;
+
+ // Payload field containing serialized Library.
+ public static final int LIBRARY_PAYLOAD_TUPLE_FIELD_INDEX = 2;
+
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(MetadataRecordTypes.LIBRARY_RECORDTYPE);
+
+ public LibraryTupleTranslator(boolean getTuple) {
+ super(getTuple, MetadataPrimaryIndexes.LIBRARY_DATASET.getFieldCount());
+ }
+
+ @Override
+ public Library getMetadataEntytiFromTuple(ITupleReference frameTuple) throws IOException {
+ byte[] serRecord = frameTuple.getFieldData(LIBRARY_PAYLOAD_TUPLE_FIELD_INDEX);
+ int recordStartOffset = frameTuple.getFieldStart(LIBRARY_PAYLOAD_TUPLE_FIELD_INDEX);
+ int recordLength = frameTuple.getFieldLength(LIBRARY_PAYLOAD_TUPLE_FIELD_INDEX);
+ ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
+ DataInput in = new DataInputStream(stream);
+ ARecord libraryRecord = (ARecord) recordSerDes.deserialize(in);
+ return createLibraryFromARecord(libraryRecord);
+ }
+
+ private Library createLibraryFromARecord(ARecord libraryRecord) {
+ String dataverseName = ((AString) libraryRecord
+ .getValueByPos(MetadataRecordTypes.LIBRARY_ARECORD_DATAVERSENAME_FIELD_INDEX)).getStringValue();
+ String libraryName = ((AString) libraryRecord
+ .getValueByPos(MetadataRecordTypes.LIBRARY_ARECORD_NAME_FIELD_INDEX)).getStringValue();
+
+ return new Library(dataverseName, libraryName);
+ }
+
+ @Override
+ public ITupleReference getTupleFromMetadataEntity(Library library) throws IOException, MetadataException {
+ // write the key in the first 2 fields of the tuple
+ tupleBuilder.reset();
+ aString.setValue(library.getDataverseName());
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+ aString.setValue(library.getName());
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+
+ // write the pay-load in the third field of the tuple
+
+ recordBuilder.reset(MetadataRecordTypes.LIBRARY_RECORDTYPE);
+
+ // write field 0
+ fieldValue.reset();
+ aString.setValue(library.getDataverseName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.LIBRARY_ARECORD_DATAVERSENAME_FIELD_INDEX, fieldValue);
+
+ // write field 1
+ fieldValue.reset();
+ aString.setValue(library.getName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.LIBRARY_ARECORD_NAME_FIELD_INDEX, fieldValue);
+
+ // write field 2
+ fieldValue.reset();
+ aString.setValue(Calendar.getInstance().getTime().toString());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.LIBRARY_ARECORD_TIMESTAMP_FIELD_INDEX, fieldValue);
+
+ // write record
+ try {
+ recordBuilder.write(tupleBuilder.getDataOutput(), true);
+ } catch (AsterixException e) {
+ throw new MetadataException(e);
+ }
+ tupleBuilder.addFieldEndOffset();
+ tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+ return tuple;
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AbstractDatasourceAdapter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AbstractDatasourceAdapter.java
new file mode 100644
index 0000000..9e8e5f7
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AbstractDatasourceAdapter.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * Represents the base class that is required to be extended by every
+ * implementation of the IDatasourceAdapter interface.
+ */
+public abstract class AbstractDatasourceAdapter implements IDatasourceAdapter {
+
+ private static final long serialVersionUID = 1L;
+
+ protected Map<String, Object> configuration;
+ protected transient AlgebricksPartitionConstraint partitionConstraint;
+ protected IAType atype;
+ protected IHyracksTaskContext ctx;
+
+ protected static final Map<String, Object> formatToParserFactoryMap = initializeFormatParserFactoryMap();
+
+ public static final String KEY_FORMAT = "format";
+ public static final String KEY_PARSER_FACTORY = "parser";
+ public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
+ public static final String FORMAT_ADM = "adm";
+
+ private static Map<String, Object> initializeFormatParserFactoryMap() {
+ Map<String, Object> map = new HashMap<String, Object>();
+ map.put(FORMAT_DELIMITED_TEXT, "edu.uci.ics.asterix.runtime.operators.file.NtDelimitedDataTupleParserFactory");
+ map.put(FORMAT_ADM, "edu.uci.ics.asterix.runtime.operators.file.AdmSchemafullRecordParserFactory");
+ return map;
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AbstractFeedDatasourceAdapter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AbstractFeedDatasourceAdapter.java
new file mode 100644
index 0000000..4921920
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AbstractFeedDatasourceAdapter.java
@@ -0,0 +1,18 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+
+public abstract class AbstractFeedDatasourceAdapter implements IDatasourceAdapter {
+
+ private static final long serialVersionUID = 1L;
+
+ protected FeedPolicyEnforcer policyEnforcer;
+
+ public FeedPolicyEnforcer getPolicyEnforcer() {
+ return policyEnforcer;
+ }
+
+ public void setFeedPolicyEnforcer(FeedPolicyEnforcer policyEnforcer) {
+ this.policyEnforcer = policyEnforcer;
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/AdapterIdentifier.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterIdentifier.java
similarity index 96%
rename from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/AdapterIdentifier.java
rename to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterIdentifier.java
index f6164ea..897faae 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/AdapterIdentifier.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterIdentifier.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.external.dataset.adapter;
+package edu.uci.ics.asterix.metadata.feeds;
import java.io.Serializable;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterRuntimeManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterRuntimeManager.java
new file mode 100644
index 0000000..44ad1e9
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterRuntimeManager.java
@@ -0,0 +1,200 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.metadata.feeds.FeedFrameWriter.Mode;
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class AdapterRuntimeManager implements IAdapterExecutor {
+
+ private static final Logger LOGGER = Logger.getLogger(AdapterRuntimeManager.class.getName());
+
+ private final FeedConnectionId feedId;
+
+ private IFeedAdapter feedAdapter;
+
+ private AdapterExecutor adapterExecutor;
+
+ private State state;
+
+ private int partition;
+
+ private IngestionRuntime ingestionRuntime;
+
+ public enum State {
+ /*
+ * Indicates that data from external source will be pushed downstream for storage
+ */
+ ACTIVE_INGESTION,
+ /*
+ * Indicates that data from external source would be buffered and not pushed downstream
+ */
+ INACTIVE_INGESTION,
+ /*
+ * Indicates that feed ingestion activity has finished
+ */
+ FINISHED_INGESTION
+ }
+
+ public AdapterRuntimeManager(FeedConnectionId feedId, IFeedAdapter feedAdapter, FeedFrameWriter writer,
+ int partition, LinkedBlockingQueue<IFeedMessage> inbox) {
+ this.feedId = feedId;
+ this.feedAdapter = feedAdapter;
+ this.partition = partition;
+ this.adapterExecutor = new AdapterExecutor(partition, writer, feedAdapter, this);
+ }
+
+ @Override
+ public void start() throws Exception {
+ state = State.ACTIVE_INGESTION;
+ ingestionRuntime = new IngestionRuntime(feedId, partition, FeedRuntimeType.INGESTION, this);
+ FeedManager.INSTANCE.registerFeedRuntime(ingestionRuntime);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Registered feed runtime manager for " + this.getFeedId());
+ }
+ ExecutorService executorService = FeedManager.INSTANCE.getFeedExecutorService(feedId);
+ executorService.execute(adapterExecutor);
+ }
+
+ @Override
+ public void stop() {
+ try {
+ feedAdapter.stop();
+ state = State.FINISHED_INGESTION;
+ synchronized (this) {
+ notifyAll();
+ }
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to stop adapter");
+ }
+ }
+ }
+
+ @Override
+ public FeedConnectionId getFeedId() {
+ return feedId;
+ }
+
+ @Override
+ public String toString() {
+ return feedId + "[" + partition + "]";
+ }
+
+ public IFeedAdapter getFeedAdapter() {
+ return feedAdapter;
+ }
+
+ public void setFeedAdapter(IFeedAdapter feedAdapter) {
+ this.feedAdapter = feedAdapter;
+ }
+
+ public static class AdapterExecutor implements Runnable {
+
+ private FeedFrameWriter writer;
+
+ private IFeedAdapter adapter;
+
+ private AdapterRuntimeManager runtimeManager;
+
+ public AdapterExecutor(int partition, FeedFrameWriter writer, IFeedAdapter adapter,
+ AdapterRuntimeManager adapterRuntimeMgr) {
+ this.writer = writer;
+ this.adapter = adapter;
+ this.runtimeManager = adapterRuntimeMgr;
+ }
+
+ @Override
+ public void run() {
+ try {
+ int partition = runtimeManager.getPartition();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Starting ingestion for partition:" + partition);
+ }
+ adapter.start(partition, writer);
+ runtimeManager.setState(State.FINISHED_INGESTION);
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("Exception during feed ingestion " + e.getMessage());
+ }
+ e.printStackTrace();
+ } finally {
+ synchronized (runtimeManager) {
+ runtimeManager.notifyAll();
+ }
+ }
+ }
+
+ public FeedFrameWriter getWriter() {
+ return writer;
+ }
+
+ public void setWriter(IFrameWriter writer) {
+ if (this.writer != null) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Switching writer to:" + writer + " from " + this.writer);
+ }
+ this.writer.setWriter(writer);
+ }
+ }
+
+ }
+
+ public synchronized State getState() {
+ return state;
+ }
+
+ @SuppressWarnings("incomplete-switch")
+ public synchronized void setState(State state) throws HyracksDataException {
+ if (this.state.equals(state)) {
+ return;
+ }
+ switch (state) {
+ case INACTIVE_INGESTION:
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Set " + Mode.STORE + " mode");
+ }
+ adapterExecutor.getWriter().setMode(Mode.STORE);
+ break;
+ case ACTIVE_INGESTION:
+ adapterExecutor.getWriter().setMode(Mode.FORWARD);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Set " + Mode.FORWARD + " mode");
+ }
+ break;
+ }
+ this.state = state;
+ }
+
+ public AdapterExecutor getAdapterExecutor() {
+ return adapterExecutor;
+ }
+
+ public int getPartition() {
+ return partition;
+ }
+
+ public IngestionRuntime getIngestionRuntime() {
+ return ingestionRuntime;
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/BuiltinFeedPolicies.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/BuiltinFeedPolicies.java
new file mode 100644
index 0000000..3db5916
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/BuiltinFeedPolicies.java
@@ -0,0 +1,107 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
+import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
+
+public class BuiltinFeedPolicies {
+
+ public static final FeedPolicy BRITTLE = initializeBrittlePolicy();
+
+ public static final FeedPolicy BASIC = initializeBasicPolicy();
+
+ public static final FeedPolicy BASIC_MONITORED = initializeBasicMonitoredPolicy();
+
+ public static final FeedPolicy FAULT_TOLERANT_BASIC_MONITORED = initializeFaultTolerantBasicMonitoredPolicy();
+
+ public static final FeedPolicy ELASTIC = initializeFaultTolerantBasicMonitoredElasticPolicy();
+
+ public static final FeedPolicy[] policies = new FeedPolicy[] { BRITTLE, BASIC, BASIC_MONITORED,
+ FAULT_TOLERANT_BASIC_MONITORED, ELASTIC };
+
+ public static final FeedPolicy DEFAULT_POLICY = FAULT_TOLERANT_BASIC_MONITORED;
+
+ public static final String CONFIG_FEED_POLICY_KEY = "policy";
+
+ public static FeedPolicy getFeedPolicy(String policyName) {
+ for (FeedPolicy policy : policies) {
+ if (policy.getPolicyName().equalsIgnoreCase(policyName)) {
+ return policy;
+ }
+ }
+ return null;
+ }
+
+ private static FeedPolicy initializeFaultTolerantBasicMonitoredElasticPolicy() {
+ Map<String, String> policyParams = new HashMap<String, String>();
+ policyParams.put(FeedPolicyAccessor.FAILURE_LOG_ERROR, "true");
+ policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_CONTINUE, "true");
+ policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_LOG_DATA, "true");
+ policyParams.put(FeedPolicyAccessor.HARDWARE_FAILURE_CONTINUE, "true");
+ policyParams.put(FeedPolicyAccessor.CLUSTER_REBOOT_AUTO_RESTART, "true");
+ policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS, "true");
+ policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS_PERIOD, "60");
+ policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS_PERIOD_UNIT, FeedPolicyAccessor.TimeUnit.SEC.name());
+ policyParams.put(FeedPolicyAccessor.ELASTIC, "true");
+ String description = "Basic Monitored Fault-Tolerant Elastic";
+ return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "BMFE", description, policyParams);
+ }
+
+ private static FeedPolicy initializeFaultTolerantBasicMonitoredPolicy() {
+ Map<String, String> policyParams = new HashMap<String, String>();
+ policyParams.put(FeedPolicyAccessor.FAILURE_LOG_ERROR, "true");
+ policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_CONTINUE, "true");
+ policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_LOG_DATA, "true");
+ policyParams.put(FeedPolicyAccessor.HARDWARE_FAILURE_CONTINUE, "true");
+ policyParams.put(FeedPolicyAccessor.CLUSTER_REBOOT_AUTO_RESTART, "true");
+ policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS, "true");
+ policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS_PERIOD, "60");
+ policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS_PERIOD_UNIT, FeedPolicyAccessor.TimeUnit.SEC.name());
+ policyParams.put(FeedPolicyAccessor.ELASTIC, "false");
+ String description = "Basic Monitored Fault-Tolerant";
+ return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "BMF", description, policyParams);
+ }
+
+ private static FeedPolicy initializeBasicMonitoredPolicy() {
+ Map<String, String> policyParams = new HashMap<String, String>();
+ policyParams.put(FeedPolicyAccessor.FAILURE_LOG_ERROR, "false");
+ policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_CONTINUE, "true");
+ policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_LOG_DATA, "true");
+ policyParams.put(FeedPolicyAccessor.HARDWARE_FAILURE_CONTINUE, "false");
+ policyParams.put(FeedPolicyAccessor.CLUSTER_REBOOT_AUTO_RESTART, "true");
+ policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS, "true");
+ policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS_PERIOD, "60");
+ policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS_PERIOD_UNIT, FeedPolicyAccessor.TimeUnit.SEC.name());
+ policyParams.put(FeedPolicyAccessor.ELASTIC, "false");
+ String description = "Basic Monitored Fault-Tolerant";
+ return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "BM", description, policyParams);
+ }
+
+ private static FeedPolicy initializeBasicPolicy() {
+ Map<String, String> policyParams = new HashMap<String, String>();
+ policyParams.put(FeedPolicyAccessor.FAILURE_LOG_ERROR, "true");
+ policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_CONTINUE, "true");
+ policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_LOG_DATA, "false");
+ policyParams.put(FeedPolicyAccessor.CLUSTER_REBOOT_AUTO_RESTART, "true");
+ policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS, "false");
+ policyParams.put(FeedPolicyAccessor.ELASTIC, "false");
+ String description = "Basic";
+ return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "B", description, policyParams);
+ }
+
+ private static FeedPolicy initializeBrittlePolicy() {
+ Map<String, String> policyParams = new HashMap<String, String>();
+ policyParams.put(FeedPolicyAccessor.FAILURE_LOG_ERROR, "false");
+ policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_CONTINUE, "false");
+ policyParams.put(FeedPolicyAccessor.APPLICATION_FAILURE_LOG_DATA, "false");
+ policyParams.put(FeedPolicyAccessor.HARDWARE_FAILURE_CONTINUE, "false");
+ policyParams.put(FeedPolicyAccessor.CLUSTER_REBOOT_AUTO_RESTART, "false");
+ policyParams.put(FeedPolicyAccessor.COLLECT_STATISTICS, "false");
+ policyParams.put(FeedPolicyAccessor.ELASTIC, "false");
+ String description = "Brittle";
+ return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "Br", description, policyParams);
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ConditionalPushTupleParserFactory.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ConditionalPushTupleParserFactory.java
new file mode 100644
index 0000000..2dfe886
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ConditionalPushTupleParserFactory.java
@@ -0,0 +1,205 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.runtime.operators.file.ADMDataParser;
+import edu.uci.ics.asterix.runtime.operators.file.AbstractTupleParser;
+import edu.uci.ics.asterix.runtime.operators.file.DelimitedDataParser;
+import edu.uci.ics.asterix.runtime.operators.file.IDataParser;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+public class ConditionalPushTupleParserFactory implements ITupleParserFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private static final Logger LOGGER = Logger.getLogger(ConditionalPushTupleParserFactory.class.getName());
+
+ @Override
+ public ITupleParser createTupleParser(IHyracksTaskContext ctx) throws HyracksDataException {
+ IDataParser dataParser = null;
+ switch (parserType) {
+ case ADM:
+ dataParser = new ADMDataParser();
+ break;
+ case DELIMITED_DATA:
+ dataParser = new DelimitedDataParser(recordType, valueParserFactories, delimiter);
+ break;
+ }
+ return new ConditionalPushTupleParser(ctx, recordType, dataParser, configuration);
+ }
+
+ private final ARecordType recordType;
+ private final Map<String, String> configuration;
+ private IValueParserFactory[] valueParserFactories;
+ private char delimiter;
+ private final ParserType parserType;
+ private Object lock;
+
+ public enum ParserType {
+ ADM,
+ DELIMITED_DATA
+ }
+
+ public ConditionalPushTupleParserFactory(ARecordType recordType, IValueParserFactory[] valueParserFactories,
+ char fieldDelimiter, Map<String, String> configuration) {
+ this.recordType = recordType;
+ this.valueParserFactories = valueParserFactories;
+ this.delimiter = fieldDelimiter;
+ this.configuration = configuration;
+ this.parserType = ParserType.DELIMITED_DATA;
+
+ }
+
+ public ConditionalPushTupleParserFactory(ARecordType recordType, Map<String, String> configuration) {
+ this.recordType = recordType;
+ this.configuration = configuration;
+ this.parserType = ParserType.ADM;
+ }
+
+}
+
+class ConditionalPushTupleParser extends AbstractTupleParser {
+
+ private final IDataParser dataParser;
+ private int batchSize;
+ private long batchInterval;
+ private boolean continueIngestion = true;
+ private int tuplesInFrame = 0;
+ private TimeBasedFlushTask flushTask;
+ private Timer timer = new Timer();
+ private Object lock = new Object();
+ private boolean activeTimer = false;
+
+ public static final String BATCH_SIZE = "batch-size";
+ public static final String BATCH_INTERVAL = "batch-interval";
+
+ public ConditionalPushTupleParser(IHyracksTaskContext ctx, ARecordType recType, IDataParser dataParser,
+ Map<String, String> configuration) throws HyracksDataException {
+ super(ctx, recType);
+ this.dataParser = dataParser;
+ String propValue = (String) configuration.get(BATCH_SIZE);
+ batchSize = propValue != null ? Integer.parseInt(propValue) : Integer.MAX_VALUE;
+ propValue = (String) configuration.get(BATCH_INTERVAL);
+ batchInterval = propValue != null ? Long.parseLong(propValue) : -1;
+ activeTimer = batchInterval > 0;
+ }
+
+ public void stop() {
+ continueIngestion = false;
+ }
+
+ @Override
+ public IDataParser getDataParser() {
+ return dataParser;
+ }
+
+ @Override
+ public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
+ flushTask = new TimeBasedFlushTask(writer, lock);
+ appender.reset(frame, true);
+ IDataParser parser = getDataParser();
+ try {
+ parser.initialize(in, recType, true);
+ if (activeTimer) {
+ timer.schedule(flushTask, 0, batchInterval);
+ }
+ while (continueIngestion) {
+ tb.reset();
+ if (!parser.parse(tb.getDataOutput())) {
+ break;
+ }
+ tb.addFieldEndOffset();
+ addTuple(writer);
+ }
+ if (appender.getTupleCount() > 0) {
+ if (activeTimer) {
+ synchronized (lock) {
+ FrameUtils.flushFrame(frame, writer);
+ }
+ } else {
+ FrameUtils.flushFrame(frame, writer);
+ }
+ }
+ } catch (AsterixException ae) {
+ throw new HyracksDataException(ae);
+ } catch (IOException ioe) {
+ throw new HyracksDataException(ioe);
+ } finally {
+ if (activeTimer) {
+ timer.cancel();
+ }
+ }
+ }
+
+ protected void addTuple(IFrameWriter writer) throws HyracksDataException {
+ if (activeTimer) {
+ synchronized (lock) {
+ addTupleToFrame(writer);
+ }
+ } else {
+ addTupleToFrame(writer);
+ }
+ }
+
+ protected void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
+ if (tuplesInFrame == batchSize || !appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ FrameUtils.flushFrame(frame, writer);
+ appender.reset(frame, true);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new IllegalStateException();
+ }
+ if (tuplesInFrame == batchSize) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Batch size exceeded! flushing frame " + "(" + tuplesInFrame + ")");
+ }
+ }
+ tuplesInFrame = 0;
+ }
+ tuplesInFrame++;
+ }
+
+ private class TimeBasedFlushTask extends TimerTask {
+
+ private IFrameWriter writer;
+ private final Object lock;
+
+ public TimeBasedFlushTask(IFrameWriter writer, Object lock) {
+ this.writer = writer;
+ this.lock = lock;
+ }
+
+ @Override
+ public void run() {
+ try {
+ if (tuplesInFrame > 0) {
+ synchronized (lock) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("TTL expired flushing frame (" + tuplesInFrame + ")");
+ }
+ FrameUtils.flushFrame(frame, writer);
+ appender.reset(frame, true);
+ tuplesInFrame = 0;
+ }
+ }
+ } catch (HyracksDataException e) {
+ e.printStackTrace();
+ }
+ }
+
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/EndFeedMessage.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/EndFeedMessage.java
new file mode 100644
index 0000000..fffee64
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/EndFeedMessage.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.feeds;
+
+/**
+ * A feed control message indicating the need to end the feed. This message is dispatched
+ * to all locations that host an operator invovled in the feed pipeline.
+ */
+public class EndFeedMessage extends FeedMessage {
+
+ private static final long serialVersionUID = 1L;
+
+ private final FeedConnectionId feedId;
+
+ public EndFeedMessage(FeedConnectionId feedId) {
+ super(MessageType.END, feedId);
+ this.feedId = feedId;
+ }
+
+ public FeedConnectionId getFeedId() {
+ return feedId;
+ }
+
+ @Override
+ public String toString() {
+ return "" + feedId;
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataScanOperatorDescriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ExternalDataScanOperatorDescriptor.java
similarity index 72%
rename from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataScanOperatorDescriptor.java
rename to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ExternalDataScanOperatorDescriptor.java
index 4290132..c5937f6 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataScanOperatorDescriptor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ExternalDataScanOperatorDescriptor.java
@@ -12,13 +12,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.external.data.operator;
+package edu.uci.ics.asterix.metadata.feeds;
-import java.util.Map;
-
-import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
-import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -36,17 +31,13 @@
private static final long serialVersionUID = 1L;
- private final Map<String, Object> adapterConfiguration;
- private final IAType atype;
- private IGenericDatasetAdapterFactory datasourceAdapterFactory;
+ private IAdapterFactory adapterFactory;
- public ExternalDataScanOperatorDescriptor(JobSpecification spec, Map<String, Object> arguments, IAType atype,
- RecordDescriptor rDesc, IGenericDatasetAdapterFactory dataSourceAdapterFactory) {
+ public ExternalDataScanOperatorDescriptor(JobSpecification spec, RecordDescriptor rDesc,
+ IAdapterFactory dataSourceAdapterFactory) {
super(spec, 0, 1);
recordDescriptors[0] = rDesc;
- this.adapterConfiguration = arguments;
- this.atype = atype;
- this.datasourceAdapterFactory = dataSourceAdapterFactory;
+ this.adapterFactory = dataSourceAdapterFactory;
}
@Override
@@ -60,9 +51,7 @@
writer.open();
IDatasourceAdapter adapter = null;
try {
- adapter = ((IGenericDatasetAdapterFactory) datasourceAdapterFactory).createAdapter(
- adapterConfiguration, atype);
- adapter.initialize(ctx);
+ adapter = ((IAdapterFactory) adapterFactory).createAdapter(ctx);
adapter.start(partition, writer);
} catch (Exception e) {
throw new HyracksDataException("exception during reading from external data source", e);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedActivityIdFactory.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedActivityIdFactory.java
new file mode 100644
index 0000000..db1a62a
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedActivityIdFactory.java
@@ -0,0 +1,25 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class FeedActivityIdFactory {
+ private static AtomicInteger id = new AtomicInteger();
+ private static boolean isInitialized = false;
+
+ public static boolean isInitialized() {
+ return isInitialized;
+ }
+
+ public static void initialize(int initialId) {
+ id.set(initialId);
+ isInitialized = true;
+ }
+
+ public static int generateFeedActivityId() {
+ return id.incrementAndGet();
+ }
+
+ public static int getMostRecentFeedActivityId() {
+ return id.get();
+ }
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedConnectionId.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedConnectionId.java
new file mode 100644
index 0000000..a487168
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedConnectionId.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.Serializable;
+
+/**
+ * A unique identifier for a data feed flowing into a dataset.
+ */
+public class FeedConnectionId implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ private final String dataverse;
+ private final String feedName;
+ private final String datasetName;
+
+ public FeedConnectionId(String dataverse, String feedName, String datasetName) {
+ this.dataverse = dataverse;
+ this.feedName = feedName;
+ this.datasetName = datasetName;
+ }
+
+ public String getDataverse() {
+ return dataverse;
+ }
+
+ public String getFeedName() {
+ return feedName;
+ }
+
+ public String getDatasetName() {
+ return datasetName;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o == null || !(o instanceof FeedConnectionId)) {
+ return false;
+ }
+ if (((FeedConnectionId) o).getFeedName().equals(feedName)
+ && ((FeedConnectionId) o).getDataverse().equals(dataverse)
+ && ((FeedConnectionId) o).getDatasetName().equals(datasetName)) {
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ return toString().hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return dataverse + "." + feedName + "-->" + datasetName;
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedFrameWriter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedFrameWriter.java
new file mode 100644
index 0000000..53038d2
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedFrameWriter.java
@@ -0,0 +1,343 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.metadata.feeds.SuperFeedManager.FeedReportMessageType;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+/**
+ * A wrapper around the standard frame writer provided to an operator node pushable.
+ * The wrapper monitors the flow of data from this operator to a downstream operator
+ * over a connector. It collects statistics if required by the feed ingestion policy
+ * and reports them to the Super Feed Manager chosen for the feed. In addition any
+ * congestion experienced by the operator is also reported.
+ */
+public class FeedFrameWriter implements IFrameWriter {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedFrameWriter.class.getName());
+
+ /** The threshold for the time required in pushing a frame to the network. **/
+ public static final long FLUSH_THRESHOLD_TIME = 5000; // 3 seconds
+
+ /** Actual frame writer provided to an operator. **/
+ private IFrameWriter writer;
+
+ /** The node pushable associated with the operator **/
+ private IOperatorNodePushable nodePushable;
+
+ /** set to true if health need to be monitored **/
+ private final boolean reportHealth;
+
+ /** A buffer for keeping frames that are waiting to be processed **/
+ private List<ByteBuffer> frames = new ArrayList<ByteBuffer>();
+
+ /**
+ * Mode associated with the frame writer
+ * Possible values: FORWARD, STORE
+ *
+ * @see Mode
+ */
+ private Mode mode;
+
+ /**
+ * Detects if the operator is unable to push a frame downstream
+ * within a threshold period of time. In addition, measure the
+ * throughput as observed on the output channel of the associated operator.
+ */
+ private HealthMonitor healthMonitor;
+
+ /**
+ * Manager scheduling of tasks
+ */
+ private Timer timer;
+
+ /**
+ * Provides access to the tuples in a frame. Used in collecting statistics.
+ */
+ private FrameTupleAccessor fta;
+
+ public enum Mode {
+ /**
+ * **
+ * Normal mode of operation for an operator when
+ * frames are pushed to the downstream operator.
+ */
+ FORWARD,
+
+ /**
+ * Failure mode of operation for an operator when
+ * input frames are not pushed to the downstream operator but
+ * are buffered for future retrieval.
+ */
+ STORE
+ }
+
+ public FeedFrameWriter(IFrameWriter writer, IOperatorNodePushable nodePushable, FeedConnectionId feedId,
+ FeedPolicyEnforcer policyEnforcer, String nodeId, FeedRuntimeType feedRuntimeType, int partition,
+ FrameTupleAccessor fta) {
+ this.writer = writer;
+ this.mode = Mode.FORWARD;
+ this.nodePushable = nodePushable;
+ this.reportHealth = policyEnforcer.getFeedPolicyAccessor().collectStatistics();
+ if (reportHealth) {
+ timer = new Timer();
+ healthMonitor = new HealthMonitor(FLUSH_THRESHOLD_TIME, feedId, nodeId, feedRuntimeType, partition,
+ FLUSH_THRESHOLD_TIME, timer);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Statistics collection enabled for the feed " + feedId + " " + feedRuntimeType + " ["
+ + partition + "]");
+ }
+ timer.scheduleAtFixedRate(healthMonitor, 0, FLUSH_THRESHOLD_TIME);
+ } else {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Statistics collection *not* enabled for the feed " + feedId + " " + feedRuntimeType + " ["
+ + partition + "]");
+ }
+ }
+ this.fta = fta;
+ }
+
+ public Mode getMode() {
+ return mode;
+ }
+
+ public void setMode(Mode newMode) throws HyracksDataException {
+ if (this.mode.equals(newMode)) {
+ return;
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Switching to :" + newMode + " from " + this.mode);
+ }
+ this.mode = newMode;
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ switch (mode) {
+ case FORWARD:
+ try {
+ if (reportHealth) {
+ fta.reset(buffer);
+ healthMonitor.notifyStart();
+ writer.nextFrame(buffer);
+ healthMonitor.notifyFinish(fta.getTupleCount());
+ } else {
+ writer.nextFrame(buffer);
+ }
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("Unable to write frame " + " on behalf of " + nodePushable.getDisplayName());
+ }
+ }
+ if (frames.size() > 0) {
+ for (ByteBuffer buf : frames) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Flusing OLD frame: " + buf + " on behalf of "
+ + nodePushable.getDisplayName());
+ }
+ writer.nextFrame(buf);
+ }
+ }
+ frames.clear();
+ break;
+ case STORE:
+ ByteBuffer storageBuffer = ByteBuffer.allocate(buffer.capacity());
+ storageBuffer.put(buffer);
+ frames.add(storageBuffer);
+ storageBuffer.flip();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Stored frame for " + nodePushable);
+ }
+ break;
+ }
+ }
+
+ private static class HealthMonitor extends TimerTask {
+
+ private long startTime = -1;
+ private FramePushState state;
+ private long flushThresholdTime;
+ private static final String EOL = "\n";
+ private FeedConnectionId feedId;
+ private String nodeId;
+ private FeedRuntimeType feedRuntimeType;
+ private int partition;
+ private AtomicLong numTuplesInInterval = new AtomicLong(0);
+ private long period;
+ private boolean collectThroughput;
+ private FeedMessageService mesgService;
+
+ public HealthMonitor(long flushThresholdTime, FeedConnectionId feedId, String nodeId,
+ FeedRuntimeType feedRuntimeType, int partition, long period, Timer timer) {
+ this.flushThresholdTime = flushThresholdTime;
+ this.state = FramePushState.INTIALIZED;
+ this.feedId = feedId;
+ this.nodeId = nodeId;
+ this.feedRuntimeType = feedRuntimeType;
+ this.partition = partition;
+ this.period = period;
+ this.collectThroughput = feedRuntimeType.equals(FeedRuntimeType.INGESTION);
+ }
+
+ public void notifyStart() {
+ startTime = System.currentTimeMillis();
+ state = FramePushState.WAITING_FOR_FLUSH_COMPLETION;
+ }
+
+ /**
+ * Reset method is invoked when a live instance of operator needs to take
+ * over from the zombie instance from the previously failed execution
+ */
+ public void reset() {
+ mesgService = null;
+ collectThroughput = true;
+ }
+
+ public void notifyFinish(int numTuples) {
+ state = FramePushState.WAITNG_FOR_NEXT_FRAME;
+ numTuplesInInterval.set(numTuplesInInterval.get() + numTuples);
+ }
+
+ @Override
+ public void run() {
+ if (state.equals(FramePushState.WAITING_FOR_FLUSH_COMPLETION)) {
+ long currentTime = System.currentTimeMillis();
+ if (currentTime - startTime > flushThresholdTime) {
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("Congestion reported by " + feedRuntimeType + " [" + partition + "]");
+ }
+ sendReportToSFM(currentTime - startTime, FeedReportMessageType.CONGESTION,
+ System.currentTimeMillis());
+ }
+ }
+ if (collectThroughput) {
+ int instantTput = (int) Math.ceil((((double) numTuplesInInterval.get() * 1000) / period));
+ System.out.println("MEASURED TPUT:" + numTuplesInInterval.get());
+ sendReportToSFM(instantTput, FeedReportMessageType.THROUGHPUT, System.currentTimeMillis());
+ }
+ numTuplesInInterval.set(0);
+ }
+
+ private void sendReportToSFM(long value, SuperFeedManager.FeedReportMessageType mesgType, long timestamp) {
+ if (mesgService == null) {
+ waitTillMessageServiceIsUp();
+ }
+ String feedRep = feedId.getDataverse() + ":" + feedId.getFeedName() + ":" + feedId.getDatasetName();
+ String message = mesgType.name().toLowerCase() + "|" + feedRep + "|" + feedRuntimeType + "|" + partition
+ + "|" + value + "|" + nodeId + "|" + timestamp + "|" + EOL;
+ try {
+ mesgService.sendMessage(message);
+ } catch (IOException ioe) {
+ ioe.printStackTrace();
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to send feed report to SFM for feed " + feedId + " " + feedRuntimeType + "["
+ + partition + "]");
+ }
+ }
+ }
+
+ private void waitTillMessageServiceIsUp() {
+ while (mesgService == null) {
+ mesgService = FeedManager.INSTANCE.getFeedMessageService(feedId);
+ if (mesgService == null) {
+ try {
+ /**
+ * wait for the message service to be available
+ */
+ Thread.sleep(2000);
+ } catch (InterruptedException e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Encountered an interrupted exception " + " Exception " + e);
+ }
+ }
+ }
+ }
+ }
+
+ public void deactivate() {
+ this.cancel();
+ collectThroughput = false;
+ }
+
+ private enum FramePushState {
+ /**
+ * Frame writer has been initialized
+ */
+ INTIALIZED,
+
+ /**
+ * Frame writer is waiting for a pending flush to finish.
+ */
+ WAITING_FOR_FLUSH_COMPLETION,
+
+ /**
+ * Frame writer is waiting to be given the next frame.
+ */
+ WAITNG_FOR_NEXT_FRAME
+ }
+
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ writer.fail();
+ if (healthMonitor != null && !healthMonitor.feedRuntimeType.equals(FeedRuntimeType.INGESTION)) {
+ healthMonitor.deactivate();
+ }
+ healthMonitor.reset();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ if (healthMonitor != null) {
+ healthMonitor.deactivate();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Closing frame statistics collection activity" + healthMonitor);
+ }
+ }
+ writer.close();
+ }
+
+ public IFrameWriter getWriter() {
+ return writer;
+ }
+
+ public void setWriter(IFrameWriter writer) {
+ this.writer = writer;
+ }
+
+ @Override
+ public String toString() {
+ return "MaterializingFrameWriter using " + writer;
+ }
+
+ public List<ByteBuffer> getStoredFrames() {
+ return frames;
+ }
+
+ public void clear() {
+ frames.clear();
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ writer.open();
+ }
+
+ public void reset() {
+ healthMonitor.reset();
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java
new file mode 100644
index 0000000..5c8c422
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java
@@ -0,0 +1,111 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeId;
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+
+/**
+ * FeedIntakeOperatorDescriptor is responsible for ingesting data from an external source. This
+ * operator uses a user specified for a built-in adaptor for retrieving data from the external
+ * data source.
+ */
+public class FeedIntakeOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ private static final Logger LOGGER = Logger.getLogger(FeedIntakeOperatorDescriptor.class.getName());
+
+ /** The type associated with the ADM data output from the feed adaptor*/
+ private final IAType atype;
+
+ /** unique identifier for a feed instance.*/
+ private final FeedConnectionId feedId;
+
+ /** Map representation of policy parameters */
+ private final Map<String, String> feedPolicy;
+
+ /** The adaptor factory that is used to create an instance of the feed adaptor**/
+ private IAdapterFactory adapterFactory;
+
+ public FeedIntakeOperatorDescriptor(JobSpecification spec, FeedConnectionId feedId, IAdapterFactory adapterFactory,
+ ARecordType atype, RecordDescriptor rDesc, Map<String, String> feedPolicy) {
+ super(spec, 0, 1);
+ recordDescriptors[0] = rDesc;
+ this.adapterFactory = adapterFactory;
+ this.atype = atype;
+ this.feedId = feedId;
+ this.feedPolicy = feedPolicy;
+ }
+
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+ throws HyracksDataException {
+ IFeedAdapter adapter;
+ FeedRuntimeId feedRuntimeId = new FeedRuntimeId(FeedRuntimeType.INGESTION, feedId, partition);
+ IngestionRuntime ingestionRuntime = (IngestionRuntime) FeedManager.INSTANCE.getFeedRuntime(feedRuntimeId);
+ try {
+ if (ingestionRuntime == null) {
+ adapter = (IFeedAdapter) adapterFactory.createAdapter(ctx);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Beginning new feed:" + feedId);
+ }
+ } else {
+ adapter = ((IngestionRuntime) ingestionRuntime).getAdapterRuntimeManager().getFeedAdapter();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Resuming old feed:" + feedId);
+ }
+ }
+ } catch (Exception e) {
+ if(LOGGER.isLoggable(Level.SEVERE)){
+ LOGGER.severe("Initialization of the feed adaptor failed");
+ }
+ throw new HyracksDataException("initialization of adapter failed", e);
+ }
+ return new FeedIntakeOperatorNodePushable(ctx, feedId, adapter, feedPolicy, partition, ingestionRuntime);
+ }
+
+ public FeedConnectionId getFeedId() {
+ return feedId;
+ }
+
+ public Map<String, String> getFeedPolicy() {
+ return feedPolicy;
+ }
+
+ public IAdapterFactory getAdapterFactory() {
+ return adapterFactory;
+ }
+
+ public IAType getAtype() {
+ return atype;
+ }
+
+ public RecordDescriptor getRecordDescriptor() {
+ return recordDescriptors[0];
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java
new file mode 100644
index 0000000..9a70ca3
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java
@@ -0,0 +1,164 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.metadata.feeds.AdapterRuntimeManager.State;
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+
+/**
+ * The runtime for @see{FeedIntakeOperationDescriptor}
+ */
+public class FeedIntakeOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
+
+ private static Logger LOGGER = Logger.getLogger(FeedIntakeOperatorNodePushable.class.getName());
+
+ private IFeedAdapter adapter;
+ private final int partition;
+ private final FeedConnectionId feedId;
+ private final LinkedBlockingQueue<IFeedMessage> inbox;
+ private final Map<String, String> feedPolicy;
+ private final FeedPolicyEnforcer policyEnforcer;
+ private FeedRuntime ingestionRuntime;
+ private final String nodeId;
+ private FrameTupleAccessor fta;
+ private FeedFrameWriter feedFrameWriter;
+
+ public FeedIntakeOperatorNodePushable(IHyracksTaskContext ctx, FeedConnectionId feedId, IFeedAdapter adapter,
+ Map<String, String> feedPolicy, int partition, IngestionRuntime ingestionRuntime) {
+ this.adapter = adapter;
+ this.partition = partition;
+ this.feedId = feedId;
+ this.ingestionRuntime = ingestionRuntime;
+ inbox = new LinkedBlockingQueue<IFeedMessage>();
+ this.feedPolicy = feedPolicy;
+ policyEnforcer = new FeedPolicyEnforcer(feedId, feedPolicy);
+ nodeId = ctx.getJobletContext().getApplicationContext().getNodeId();
+ fta = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
+
+ }
+
+ @Override
+ public void initialize() throws HyracksDataException {
+
+ AdapterRuntimeManager adapterRuntimeMgr = null;
+ try {
+ if (ingestionRuntime == null) {
+ feedFrameWriter = new FeedFrameWriter(writer, this, feedId, policyEnforcer, nodeId,
+ FeedRuntimeType.INGESTION, partition, fta);
+ adapterRuntimeMgr = new AdapterRuntimeManager(feedId, adapter, feedFrameWriter, partition, inbox);
+
+ if (adapter instanceof AbstractFeedDatasourceAdapter) {
+ ((AbstractFeedDatasourceAdapter) adapter).setFeedPolicyEnforcer(policyEnforcer);
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Beginning new feed:" + feedId);
+ }
+ feedFrameWriter.open();
+ adapterRuntimeMgr.start();
+ } else {
+ adapterRuntimeMgr = ((IngestionRuntime) ingestionRuntime).getAdapterRuntimeManager();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Resuming old feed:" + feedId);
+ }
+ adapter = adapterRuntimeMgr.getFeedAdapter();
+ writer.open();
+ adapterRuntimeMgr.getAdapterExecutor().setWriter(writer);
+ adapterRuntimeMgr.getAdapterExecutor().getWriter().reset();
+ adapterRuntimeMgr.setState(State.ACTIVE_INGESTION);
+ feedFrameWriter = adapterRuntimeMgr.getAdapterExecutor().getWriter();
+ }
+
+ ingestionRuntime = adapterRuntimeMgr.getIngestionRuntime();
+ synchronized (adapterRuntimeMgr) {
+ while (!adapterRuntimeMgr.getState().equals(State.FINISHED_INGESTION)) {
+ adapterRuntimeMgr.wait();
+ }
+ }
+ FeedManager.INSTANCE.deRegisterFeedRuntime(ingestionRuntime.getFeedRuntimeId());
+ feedFrameWriter.close();
+ } catch (InterruptedException ie) {
+ if (policyEnforcer.getFeedPolicyAccessor().continueOnHardwareFailure()) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Continuing on failure as per feed policy, switching to INACTIVE INGESTION temporarily");
+ }
+ adapterRuntimeMgr.setState(State.INACTIVE_INGESTION);
+ FeedRuntimeManager runtimeMgr = FeedManager.INSTANCE.getFeedRuntimeManager(feedId);
+ try {
+ runtimeMgr.close(false);
+ } catch (IOException ioe) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to close Feed Runtime Manager " + ioe.getMessage());
+ }
+ }
+ feedFrameWriter.fail();
+ } else {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Interrupted Exception, something went wrong");
+ }
+
+ FeedManager.INSTANCE.deRegisterFeedRuntime(ingestionRuntime.getFeedRuntimeId());
+ feedFrameWriter.close();
+ throw new HyracksDataException(ie);
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new HyracksDataException(e);
+ }
+ }
+
+ public Map<String, String> getFeedPolicy() {
+ return feedPolicy;
+ }
+}
+
+class FeedInboxMonitor extends Thread {
+
+ private LinkedBlockingQueue<IFeedMessage> inbox;
+ private final AdapterRuntimeManager runtimeMgr;
+
+ public FeedInboxMonitor(AdapterRuntimeManager runtimeMgr, LinkedBlockingQueue<IFeedMessage> inbox, int partition) {
+ this.inbox = inbox;
+ this.runtimeMgr = runtimeMgr;
+ }
+
+ @Override
+ public void run() {
+ while (true) {
+ try {
+ IFeedMessage feedMessage = inbox.take();
+ switch (feedMessage.getMessageType()) {
+ case END:
+ runtimeMgr.stop();
+ break;
+ }
+ } catch (InterruptedException ie) {
+ break;
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedManager.java
new file mode 100644
index 0000000..d910ca8
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedManager.java
@@ -0,0 +1,129 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeId;
+
+public class FeedManager implements IFeedManager {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedManager.class.getName());
+
+ public static final long SOCKET_CONNECT_TIMEOUT = 5000;
+
+ public static FeedManager INSTANCE = new FeedManager();
+
+ private FeedManager() {
+
+ }
+
+ private Map<FeedConnectionId, FeedRuntimeManager> feedRuntimeManagers = new HashMap<FeedConnectionId, FeedRuntimeManager>();
+
+ public FeedRuntimeManager getFeedRuntimeManager(FeedConnectionId feedId) {
+ return feedRuntimeManagers.get(feedId);
+ }
+
+ public ExecutorService getFeedExecutorService(FeedConnectionId feedId) {
+ FeedRuntimeManager mgr = feedRuntimeManagers.get(feedId);
+ return mgr == null ? null : mgr.getExecutorService();
+ }
+
+ public FeedMessageService getFeedMessageService(FeedConnectionId feedId) {
+ FeedRuntimeManager mgr = feedRuntimeManagers.get(feedId);
+ return mgr == null ? null : mgr.getMessageService();
+ }
+
+ @Override
+ public void deregisterFeed(FeedConnectionId feedId) {
+ try {
+ FeedRuntimeManager mgr = feedRuntimeManagers.get(feedId);
+ if (mgr == null) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("unknown feed id: " + feedId);
+ }
+ } else {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Closing feed runtime manager: " + mgr);
+ }
+ mgr.close(true);
+ }
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Exception in closing feed runtime" + e.getMessage());
+ }
+ e.printStackTrace();
+ }
+
+ feedRuntimeManagers.remove(feedId);
+ }
+
+ @Override
+ public void registerFeedRuntime(FeedRuntime feedRuntime) throws Exception {
+ FeedConnectionId feedId = feedRuntime.getFeedRuntimeId().getFeedId();
+ FeedRuntimeManager runtimeMgr = feedRuntimeManagers.get(feedId);
+ if (runtimeMgr == null) {
+ synchronized (feedRuntimeManagers) {
+ if (runtimeMgr == null) {
+ runtimeMgr = new FeedRuntimeManager(feedId);
+ feedRuntimeManagers.put(feedId, runtimeMgr);
+ }
+ }
+ }
+
+ runtimeMgr.registerFeedRuntime(feedRuntime.getFeedRuntimeId(), feedRuntime);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Registered runtime " + feedRuntime + " for feed " + feedId);
+ }
+ }
+
+ @Override
+ public void deRegisterFeedRuntime(FeedRuntimeId feedRuntimeId) {
+ FeedRuntimeManager runtimeMgr = feedRuntimeManagers.get(feedRuntimeId.getFeedId());
+ if (runtimeMgr != null) {
+ runtimeMgr.deregisterFeedRuntime(feedRuntimeId);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Deregistered Feed Runtime " + feedRuntimeId);
+ }
+ }
+ }
+
+ @Override
+ public FeedRuntime getFeedRuntime(FeedRuntimeId feedRuntimeId) {
+ FeedRuntimeManager runtimeMgr = feedRuntimeManagers.get(feedRuntimeId.getFeedId());
+ return runtimeMgr != null ? runtimeMgr.getFeedRuntime(feedRuntimeId) : null;
+ }
+
+ @Override
+ public void registerSuperFeedManager(FeedConnectionId feedId, SuperFeedManager sfm) throws Exception {
+ FeedRuntimeManager runtimeMgr = feedRuntimeManagers.get(feedId);
+ if (runtimeMgr != null) {
+ runtimeMgr.setSuperFeedManager(sfm);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Registered Super Feed Manager " + sfm);
+ }
+ }
+ }
+
+ @Override
+ public SuperFeedManager getSuperFeedManager(FeedConnectionId feedId) {
+ FeedRuntimeManager runtimeMgr = feedRuntimeManagers.get(feedId);
+ return runtimeMgr != null ? runtimeMgr.getSuperFeedManager() : null;
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedManagerElectMessage.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedManagerElectMessage.java
new file mode 100644
index 0000000..aedd124
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedManagerElectMessage.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.feeds;
+
+/**
+ * A feed control message containing the altered values for
+ * adapter configuration parameters. This message is dispatched
+ * to all runtime instances of the feed's adapter.
+ */
+public class FeedManagerElectMessage extends FeedMessage {
+
+ private static final long serialVersionUID = 1L;
+
+ private final String host;
+ private final String nodeId;
+ private final int port;
+
+ public FeedManagerElectMessage(String host, String nodeId, int port, FeedConnectionId feedId) {
+ super(MessageType.SUPER_FEED_MANAGER_ELECT, feedId);
+ this.host = host;
+ this.port = port;
+ this.nodeId = nodeId;
+ }
+
+ @Override
+ public MessageType getMessageType() {
+ return MessageType.SUPER_FEED_MANAGER_ELECT;
+ }
+
+ @Override
+ public String toString() {
+ return host + "_" + nodeId + "[" + port + "]";
+ }
+
+ public String getHost() {
+ return host;
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ public int getPort() {
+ return port;
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/FeedMessage.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessage.java
similarity index 71%
rename from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/FeedMessage.java
rename to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessage.java
index 96262fe..36fcf4b 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/FeedMessage.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessage.java
@@ -12,28 +12,32 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.external.feed.lifecycle;
+package edu.uci.ics.asterix.metadata.feeds;
+
+import edu.uci.ics.asterix.metadata.feeds.IFeedMessage;
/**
- * A control message that can be sent to the runtime instance of a
- * feed's adapter.
+ * A control message that can be sent to the runtime instance of a
+ * feed's adapter.
*/
public class FeedMessage implements IFeedMessage {
private static final long serialVersionUID = 1L;
- protected MessageType messageType;
+ protected final MessageType messageType;
+ protected final FeedConnectionId feedId;
- public FeedMessage(MessageType messageType) {
+ public FeedMessage(MessageType messageType, FeedConnectionId feedId) {
this.messageType = messageType;
+ this.feedId = feedId;
}
public MessageType getMessageType() {
return messageType;
}
- public void setMessageType(MessageType messageType) {
- this.messageType = messageType;
+ public FeedConnectionId getFeedId() {
+ return feedId;
}
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedMessageOperatorDescriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorDescriptor.java
similarity index 74%
rename from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedMessageOperatorDescriptor.java
rename to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorDescriptor.java
index 300e3af..fae2fe4 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedMessageOperatorDescriptor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorDescriptor.java
@@ -12,12 +12,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.external.data.operator;
+package edu.uci.ics.asterix.metadata.feeds;
-import java.util.List;
-
-import edu.uci.ics.asterix.external.feed.lifecycle.FeedId;
-import edu.uci.ics.asterix.external.feed.lifecycle.IFeedMessage;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -32,21 +28,22 @@
private static final long serialVersionUID = 1L;
- private final FeedId feedId;
- private final List<IFeedMessage> feedMessages;
- private final boolean sendToAll = true;
+ private final FeedConnectionId feedId;
+ private final IFeedMessage feedMessage;
- public FeedMessageOperatorDescriptor(JobSpecification spec, String dataverse, String dataset,
- List<IFeedMessage> feedMessages) {
+ public FeedMessageOperatorDescriptor(JobSpecification spec, String dataverse, String feedName, String dataset,
+ IFeedMessage feedMessage) {
super(spec, 0, 1);
- this.feedId = new FeedId(dataverse, dataset);
- this.feedMessages = feedMessages;
+ this.feedId = new FeedConnectionId(dataverse, feedName, dataset);
+ this.feedMessage = feedMessage;
}
@Override
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
- return new FeedMessageOperatorNodePushable(ctx, feedId, feedMessages, sendToAll, partition, nPartitions);
+ return new FeedMessageOperatorNodePushable(ctx, feedId, feedMessage, partition, nPartitions);
}
+
+
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorNodePushable.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorNodePushable.java
new file mode 100644
index 0000000..a8541f5
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorNodePushable.java
@@ -0,0 +1,101 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeId;
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.hyracks.api.application.INCApplicationContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+
+/**
+ * Runtime for the @see{FeedMessageOperatorDescriptor}
+ */
+public class FeedMessageOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedMessageOperatorNodePushable.class.getName());
+
+ private final FeedConnectionId feedId;
+ private final IFeedMessage feedMessage;
+ private final int partition;
+ private final IHyracksTaskContext ctx;
+
+ public FeedMessageOperatorNodePushable(IHyracksTaskContext ctx, FeedConnectionId feedId, IFeedMessage feedMessage,
+ int partition, int nPartitions) {
+ this.feedId = feedId;
+ this.feedMessage = feedMessage;
+ this.partition = partition;
+ this.ctx = ctx;
+ }
+
+ @Override
+ public void initialize() throws HyracksDataException {
+ try {
+ writer.open();
+ FeedRuntimeId runtimeId = new FeedRuntimeId(FeedRuntimeType.INGESTION, feedId, partition);
+ FeedRuntime feedRuntime = FeedManager.INSTANCE.getFeedRuntime(runtimeId);
+ boolean ingestionLocation = feedRuntime != null;
+
+ switch (feedMessage.getMessageType()) {
+ case END:
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Ending feed:" + feedId);
+ }
+
+ if (ingestionLocation) {
+ AdapterRuntimeManager adapterRuntimeMgr = ((IngestionRuntime) feedRuntime)
+ .getAdapterRuntimeManager();
+ adapterRuntimeMgr.stop();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Terminating ingestion for :" + feedId);
+ }
+ }
+ break;
+
+ case SUPER_FEED_MANAGER_ELECT:
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Registering SUPER Feed MGR for :" + feedId);
+ }
+ FeedManagerElectMessage mesg = ((FeedManagerElectMessage) feedMessage);
+ SuperFeedManager sfm = new SuperFeedManager(mesg.getFeedId(), mesg.getHost(), mesg.getNodeId(),
+ mesg.getPort());
+ synchronized (FeedManager.INSTANCE) {
+ INCApplicationContext ncCtx = ctx.getJobletContext().getApplicationContext();
+ String nodeId = ncCtx.getNodeId();
+ if (sfm.getNodeId().equals(nodeId)) {
+ sfm.setLocal(true);
+ } else {
+ Thread.sleep(5000);
+ }
+ FeedManager.INSTANCE.registerSuperFeedManager(feedId, sfm);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Registered super feed mgr " + sfm + " for feed " + feedId);
+ }
+ }
+ break;
+ }
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new HyracksDataException(e);
+ } finally {
+ writer.close();
+ }
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageService.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageService.java
new file mode 100644
index 0000000..40b2f26
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageService.java
@@ -0,0 +1,143 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.Socket;
+import java.net.UnknownHostException;
+import java.nio.CharBuffer;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * Sends feed report messages on behalf of an operator instance
+ * to the SuperFeedMaanger associated with the feed.
+ */
+public class FeedMessageService {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedMessageService.class.getName());
+ private static final char EOL = (char) "\n".getBytes()[0];
+
+ private final FeedConnectionId feedId;
+ private final LinkedBlockingQueue<String> inbox;
+ private final FeedMessageHandler mesgHandler;
+
+ public FeedMessageService(FeedConnectionId feedId) {
+ this.feedId = feedId;
+ this.inbox = new LinkedBlockingQueue<String>();
+ mesgHandler = new FeedMessageHandler(inbox, feedId);
+ }
+
+ public void start() throws UnknownHostException, IOException, Exception {
+ FeedManager.INSTANCE.getFeedExecutorService(feedId).execute(mesgHandler);
+ }
+
+ public void stop() throws IOException {
+ mesgHandler.stop();
+ }
+
+ public void sendMessage(String message) throws IOException {
+ inbox.add(message);
+ }
+
+ private static class FeedMessageHandler implements Runnable {
+
+ private final LinkedBlockingQueue<String> inbox;
+ private final FeedConnectionId feedId;
+ private Socket sfmSocket;
+ private boolean process = true;
+
+ public FeedMessageHandler(LinkedBlockingQueue<String> inbox, FeedConnectionId feedId) {
+ this.inbox = inbox;
+ this.feedId = feedId;
+ }
+
+ public void run() {
+ try {
+ sfmSocket = obtainSFMSocket();
+ if (sfmSocket != null) {
+ while (process) {
+ String message = inbox.take();
+ sfmSocket.getOutputStream().write(message.getBytes());
+ }
+ } else {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to start feed message service for " + feedId);
+ }
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("ENDED FEED MESSAGE SERVICE for " + feedId);
+ }
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Exception in handling incoming feed messages" + e.getMessage());
+ }
+ } finally {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Stopping feed message handler");
+ }
+ if (sfmSocket != null) {
+ try {
+ sfmSocket.close();
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Exception in closing socket " + e.getMessage());
+ }
+ }
+ }
+ }
+
+ }
+
+ public void stop() {
+ process = false;
+ }
+
+ private Socket obtainSFMSocket() throws UnknownHostException, IOException, Exception {
+ Socket sfmDirServiceSocket = null;
+ SuperFeedManager sfm = FeedManager.INSTANCE.getSuperFeedManager(feedId);
+ try {
+ FeedRuntimeManager runtimeManager = FeedManager.INSTANCE.getFeedRuntimeManager(feedId);
+ sfmDirServiceSocket = runtimeManager.createClientSocket(sfm.getHost(), sfm.getPort(),
+ FeedManager.SOCKET_CONNECT_TIMEOUT);
+ if (sfmDirServiceSocket == null) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to connect to " + sfm.getHost() + "[" + sfm.getPort() + "]");
+ }
+ } else {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(" Connected to Super Feed Manager service " + sfm.getHost() + " " + sfm.getPort());
+ }
+ while (!sfmDirServiceSocket.isConnected()) {
+ Thread.sleep(2000);
+ }
+ InputStream in = sfmDirServiceSocket.getInputStream();
+ CharBuffer buffer = CharBuffer.allocate(50);
+ char ch = 0;
+ while (ch != EOL) {
+ buffer.put(ch);
+ ch = (char) in.read();
+ }
+ buffer.flip();
+ String s = new String(buffer.array());
+ int port = Integer.parseInt(s.trim());
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Response from Super Feed Manager service " + port + " will connect at "
+ + sfm.getHost() + " " + port);
+ }
+ sfmSocket = runtimeManager.createClientSocket(sfm.getHost(), port,
+ FeedManager.SOCKET_CONNECT_TIMEOUT);
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw e;
+ } finally {
+ if (sfmDirServiceSocket != null) {
+ sfmDirServiceSocket.close();
+ }
+ }
+ return sfmSocket;
+ }
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaOperatorDescriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaOperatorDescriptor.java
new file mode 100644
index 0000000..bcadc18
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaOperatorDescriptor.java
@@ -0,0 +1,272 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeId;
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeState;
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+
+/**
+ * FeedMetaOperatorDescriptor is a wrapper operator that provides a sanboox like
+ * environment for an hyracks operator that is part of a feed ingestion pipeline.
+ * The MetaFeed operator provides an interface iden- tical to that offered by the
+ * underlying wrapped operator, hereafter referred to as the core operator.
+ * As seen by Hyracks, the altered pipeline is identical to the earlier version formed
+ * from core operators. The MetaFeed operator enhances each core operator by providing
+ * functionality for handling runtime exceptions, saving any state for future retrieval,
+ * and measuring/reporting of performance characteristics. We next describe how the added
+ * functionality contributes to providing fault- tolerance.
+ */
+
+public class FeedMetaOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ private static final Logger LOGGER = Logger.getLogger(FeedMetaOperatorDescriptor.class.getName());
+
+ /** The actual (Hyracks) operator that is wrapped around by the Metafeed Adaptor **/
+ private IOperatorDescriptor coreOperator;
+
+ /**
+ * A unique identifier for the feed instance. A feed instance represents the flow of data
+ * from a feed to a dataset.
+ **/
+ private final FeedConnectionId feedConnectionId;
+
+ /**
+ * The policy associated with the feed instance.
+ */
+ private final FeedPolicy feedPolicy;
+
+ /**
+ * type for the feed runtime associated with the operator.
+ * Possible values: INGESTION, COMPUTE, STORAGE, COMMIT
+ */
+ private final FeedRuntimeType runtimeType;
+
+ public FeedMetaOperatorDescriptor(JobSpecification spec, FeedConnectionId feedConnectionId,
+ IOperatorDescriptor coreOperatorDescriptor, FeedPolicy feedPolicy, FeedRuntimeType runtimeType) {
+ super(spec, coreOperatorDescriptor.getInputArity(), coreOperatorDescriptor.getOutputArity());
+ this.feedConnectionId = feedConnectionId;
+ this.feedPolicy = feedPolicy;
+ if (coreOperatorDescriptor.getOutputRecordDescriptors().length == 1) {
+ recordDescriptors[0] = coreOperatorDescriptor.getOutputRecordDescriptors()[0];
+ }
+ this.coreOperator = coreOperatorDescriptor;
+ this.runtimeType = runtimeType;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+ return new FeedMetaNodePushable(ctx, recordDescProvider, partition, nPartitions, coreOperator,
+ feedConnectionId, feedPolicy, runtimeType);
+ }
+
+ @Override
+ public String toString() {
+ return "FeedMeta [" + coreOperator + " ]";
+ }
+
+ private static class FeedMetaNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+
+ /** Runtime node pushable corresponding to the core feed operator **/
+ private AbstractUnaryInputUnaryOutputOperatorNodePushable coreOperatorNodePushable;
+
+ /**
+ * A policy enforcer that ensures dyanmic decisions for a feed are taken in accordance
+ * with the associated ingestion policy
+ **/
+ private FeedPolicyEnforcer policyEnforcer;
+
+ /**
+ * The Feed Runtime instance associated with the operator. Feed Runtime captures the state of the operator while
+ * the feed is active.
+ */
+ private FeedRuntime feedRuntime;
+
+ /**
+ * A unique identifier for the feed instance. A feed instance represents the flow of data
+ * from a feed to a dataset.
+ **/
+ private FeedConnectionId feedId;
+
+ /** Denotes the i'th operator instance in a setting where K operator instances are scheduled to run in parallel **/
+ private int partition;
+
+ /** A buffer that is used to hold the current frame that is being processed **/
+ private ByteBuffer currentBuffer;
+
+ /** Type associated with the core feed operator **/
+ private final FeedRuntimeType runtimeType;
+
+ /** True is the feed is recovering from a previous failed execution **/
+ private boolean resumeOldState;
+
+ /** The Node Controller ID for the host NC **/
+
+ private String nodeId;
+
+ /** Allows to iterate over the tuples in a frame **/
+ private FrameTupleAccessor fta;
+
+ public FeedMetaNodePushable(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
+ int partition, int nPartitions, IOperatorDescriptor coreOperator, FeedConnectionId feedConnectionId,
+ FeedPolicy feedPolicy, FeedRuntimeType runtimeType) throws HyracksDataException {
+ this.coreOperatorNodePushable = (AbstractUnaryInputUnaryOutputOperatorNodePushable) ((IActivity) coreOperator)
+ .createPushRuntime(ctx, recordDescProvider, partition, nPartitions);
+ this.policyEnforcer = new FeedPolicyEnforcer(feedConnectionId, feedPolicy.getProperties());
+ this.partition = partition;
+ this.runtimeType = runtimeType;
+ this.feedId = feedConnectionId;
+ this.nodeId = ctx.getJobletContext().getApplicationContext().getNodeId();
+ fta = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ FeedRuntimeId runtimeId = new FeedRuntimeId(runtimeType, feedId, partition);
+ try {
+ feedRuntime = FeedManager.INSTANCE.getFeedRuntime(runtimeId);
+ if (feedRuntime == null) {
+ feedRuntime = new FeedRuntime(feedId, partition, runtimeType);
+ FeedManager.INSTANCE.registerFeedRuntime(feedRuntime);
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Did not find a saved state from a previous zombie, starting a new instance for "
+ + runtimeType + " node.");
+ }
+ resumeOldState = false;
+ } else {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Retreived state from the zombie instance from previous execution for "
+ + runtimeType + " node.");
+ }
+ resumeOldState = true;
+ }
+ FeedFrameWriter mWriter = new FeedFrameWriter(writer, this, feedId, policyEnforcer, nodeId,
+ runtimeType, partition, fta);
+ coreOperatorNodePushable.setOutputFrameWriter(0, mWriter, recordDesc);
+ coreOperatorNodePushable.open();
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("Unable to initialize feed operator " + feedRuntime + " [" + partition + "]");
+ }
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ try {
+ if (resumeOldState) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("State from previous zombie instance "
+ + feedRuntime.getRuntimeState().getFrame());
+ }
+ coreOperatorNodePushable.nextFrame(feedRuntime.getRuntimeState().getFrame());
+ feedRuntime.setRuntimeState(null);
+ resumeOldState = false;
+ }
+ coreOperatorNodePushable.nextFrame(buffer);
+ } catch (HyracksDataException e) {
+ if (policyEnforcer.getFeedPolicyAccessor().continueOnApplicationFailure()) {
+ boolean isExceptionHarmful = handleException(e.getCause());
+ if (isExceptionHarmful) {
+ // log the tuple
+ FeedRuntimeState runtimeState = new FeedRuntimeState(buffer, writer, e);
+ feedRuntime.setRuntimeState(runtimeState);
+ String message = e.getMessage();
+ String tIndexString = message.substring(message.lastIndexOf(':'));
+ int tupleIndex = 0;
+ if (tIndexString != null) {
+ tupleIndex = Integer.parseInt(tIndexString);
+ }
+ fta.reset(buffer);
+ int endOffset = fta.getTupleEndOffset(tupleIndex);
+ buffer.flip();
+ buffer.position(endOffset + 1);
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Harmful exception (parked data) tupleIndex " + tupleIndex + e);
+ }
+ } else {
+ // ignore the frame (exception is expected)
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Ignoring exception " + e);
+ }
+ }
+ } else {
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("Feed policy does not require feed to survive soft failure");
+ }
+ throw e;
+ }
+ }
+ }
+
+ private boolean handleException(Throwable exception) {
+ if (exception instanceof TreeIndexException) {
+ if (resumeOldState) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Received duplicate key exception but that is possible post recovery");
+ }
+ return false;
+ } else {
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("Received duplicate key exception!");
+ }
+ return true;
+ }
+ }
+ return true;
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.info("Core Op:" + coreOperatorNodePushable.getDisplayName() + " fail ");
+ }
+ if (policyEnforcer.getFeedPolicyAccessor().continueOnHardwareFailure()) {
+ if (currentBuffer != null) {
+ FeedRuntimeState runtimeState = new FeedRuntimeState(currentBuffer, writer, null);
+ feedRuntime.setRuntimeState(runtimeState);
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Saved feed compute runtime for revivals" + feedRuntime.getFeedRuntimeId());
+ }
+ } else {
+ FeedManager.INSTANCE.deRegisterFeedRuntime(feedRuntime.getFeedRuntimeId());
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning(" No state to save, de-registered feed compute runtime "
+ + feedRuntime.getFeedRuntimeId());
+ }
+ }
+ }
+ coreOperatorNodePushable.fail();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ coreOperatorNodePushable.close();
+ FeedManager.INSTANCE.deRegisterFeedRuntime(feedRuntime.getFeedRuntimeId());
+ }
+
+ }
+
+ public IOperatorDescriptor getCoreOperator() {
+ return coreOperator;
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyAccessor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyAccessor.java
new file mode 100644
index 0000000..a3c505b
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyAccessor.java
@@ -0,0 +1,91 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.Map;
+
+public class FeedPolicyAccessor {
+ public static final String FAILURE_LOG_ERROR = "failure.log.error";
+ public static final String APPLICATION_FAILURE_LOG_DATA = "application.failure.log.data";
+ public static final String APPLICATION_FAILURE_CONTINUE = "application.failure.continue";
+ public static final String HARDWARE_FAILURE_CONTINUE = "hardware.failure.continue";
+ public static final String CLUSTER_REBOOT_AUTO_RESTART = "cluster.reboot.auto.restart";
+ public static final String COLLECT_STATISTICS = "collect.statistics";
+ public static final String COLLECT_STATISTICS_PERIOD = "collect.statistics.period";
+ public static final String COLLECT_STATISTICS_PERIOD_UNIT = "collect.statistics.period.unit";
+ public static final String ELASTIC = "elastic";
+
+ public enum TimeUnit {
+ SEC,
+ MIN,
+ HRS,
+ DAYS
+ }
+
+ private Map<String, String> feedPolicy;
+
+ public FeedPolicyAccessor(Map<String, String> feedPolicy) {
+ this.feedPolicy = feedPolicy;
+ }
+
+ public boolean logErrorOnFailure() {
+ return getBooleanPropertyValue(FAILURE_LOG_ERROR);
+ }
+
+ public boolean logDataOnApplicationFailure() {
+ return getBooleanPropertyValue(APPLICATION_FAILURE_LOG_DATA);
+ }
+
+ public boolean continueOnApplicationFailure() {
+ return getBooleanPropertyValue(APPLICATION_FAILURE_CONTINUE);
+ }
+
+ public boolean continueOnHardwareFailure() {
+ return getBooleanPropertyValue(HARDWARE_FAILURE_CONTINUE);
+ }
+
+ public boolean autoRestartOnClusterReboot() {
+ return getBooleanPropertyValue(CLUSTER_REBOOT_AUTO_RESTART);
+ }
+
+ public boolean collectStatistics() {
+ return getBooleanPropertyValue(COLLECT_STATISTICS);
+ }
+
+ public long getStatisicsCollectionPeriodInSecs() {
+ return getIntegerPropertyValue(COLLECT_STATISTICS_PERIOD) * getTimeUnitFactor();
+ }
+
+ public boolean isElastic() {
+ return getBooleanPropertyValue(ELASTIC);
+ }
+
+ private int getTimeUnitFactor() {
+ String v = feedPolicy.get(COLLECT_STATISTICS_PERIOD_UNIT);
+ int factor = 1;
+ switch (TimeUnit.valueOf(v)) {
+ case SEC:
+ factor = 1;
+ break;
+ case MIN:
+ factor = 60;
+ break;
+ case HRS:
+ factor = 3600;
+ break;
+ case DAYS:
+ factor = 216000;
+ break;
+
+ }
+ return factor;
+ }
+
+ private boolean getBooleanPropertyValue(String key) {
+ String v = feedPolicy.get(key);
+ return v == null ? false : Boolean.valueOf(v);
+ }
+
+ private int getIntegerPropertyValue(String key) {
+ String v = feedPolicy.get(key);
+ return Integer.parseInt(v);
+ }
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyEnforcer.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyEnforcer.java
new file mode 100644
index 0000000..b94e228
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyEnforcer.java
@@ -0,0 +1,64 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.rmi.RemoteException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
+
+public class FeedPolicyEnforcer {
+
+ private final FeedConnectionId feedId;
+ private final FeedPolicyAccessor feedPolicyAccessor;
+ private final FeedActivity feedActivity;
+
+ public FeedPolicyEnforcer(FeedConnectionId feedId, Map<String, String> feedPolicy) {
+ this.feedId = feedId;
+ this.feedPolicyAccessor = new FeedPolicyAccessor(feedPolicy);
+ this.feedActivity = new FeedActivity(feedId.getDataverse(), feedId.getFeedName(), feedId.getDatasetName(),
+ null, new HashMap<String, String>());
+ }
+
+ public boolean handleSoftwareFailure(Exception e) throws RemoteException, ACIDException {
+ boolean continueIngestion = feedPolicyAccessor.continueOnApplicationFailure();
+ if (feedPolicyAccessor.logErrorOnFailure()) {
+ persistExceptionDetails(e);
+ }
+ return continueIngestion;
+ }
+
+ private synchronized void persistExceptionDetails(Exception e) throws RemoteException, ACIDException {
+ MetadataManager.INSTANCE.acquireWriteLatch();
+ MetadataTransactionContext ctx = null;
+ try {
+ ctx = MetadataManager.INSTANCE.beginTransaction();
+ feedActivity.setActivityType(FeedActivityType.FEED_FAILURE);
+ feedActivity.getFeedActivityDetails().put(FeedActivity.FeedActivityDetails.EXCEPTION_MESSAGE,
+ e.getMessage());
+ MetadataManager.INSTANCE.registerFeedActivity(ctx, feedId, feedActivity);
+ MetadataManager.INSTANCE.commitTransaction(ctx);
+ } catch (Exception e2) {
+ MetadataManager.INSTANCE.abortTransaction(ctx);
+ } finally {
+ MetadataManager.INSTANCE.releaseWriteLatch();
+ }
+ }
+
+ public void handleHardwareFailure(List<String> nodeId) {
+
+ }
+
+ public FeedPolicyAccessor getFeedPolicyAccessor() {
+ return feedPolicyAccessor;
+ }
+
+ public FeedConnectionId getFeedId() {
+ return feedId;
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedReport.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedReport.java
new file mode 100644
index 0000000..fbff931
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedReport.java
@@ -0,0 +1,116 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.metadata.feeds.SuperFeedManager.FeedReportMessageType;
+
+public class FeedReport implements Comparable {
+
+ private FeedConnectionId feedId;
+ private FeedReportMessageType reportType;
+ private int partition = -1;
+ private FeedRuntimeType runtimeType;
+ private long value = -1;
+ private String[] representation;
+
+ public FeedReport() {
+ }
+
+ public FeedReport(String message) {
+ representation = message.split("\\|");
+ }
+
+ public void reset(String message) {
+ representation = message.split("\\|");
+ reportType = null;
+ feedId = null;
+ runtimeType = null;
+ partition = -1;
+ value = -1;
+ }
+
+ @Override
+ public String toString() {
+ return getFeedId() + " " + getReportType() + " " + getPartition() + " " + getRuntimeType() + " " + getValue();
+ }
+
+ public FeedConnectionId getFeedId() {
+ if (feedId == null) {
+ String feedIdRep = representation[1];
+ String[] feedIdComp = feedIdRep.split(":");
+ feedId = new FeedConnectionId(feedIdComp[0], feedIdComp[1], feedIdComp[2]);
+ }
+ return feedId;
+ }
+
+ public FeedReportMessageType getReportType() {
+ if (reportType == null) {
+ reportType = FeedReportMessageType.valueOf(representation[0].toUpperCase());
+ }
+ return reportType;
+ }
+
+ public int getPartition() {
+ if (partition < 0) {
+ partition = Integer.parseInt(representation[3]);
+ }
+ return partition;
+ }
+
+ public FeedRuntimeType getRuntimeType() {
+ if (runtimeType == null) {
+ runtimeType = FeedRuntimeType.valueOf(representation[2].toUpperCase());
+ }
+ return runtimeType;
+ }
+
+ public long getValue() {
+ if (value < 0) {
+ value = Long.parseLong(representation[4]);
+ }
+ return value;
+ }
+
+ public String[] getRepresentation() {
+ return representation;
+ }
+
+ @Override
+ public int compareTo(Object o) {
+ if (!(o instanceof FeedReport)) {
+ throw new IllegalArgumentException("Incorrect operand type " + o);
+ }
+
+ FeedReport other = (FeedReport) o;
+ if (!other.getReportType().equals(getReportType())) {
+ throw new IllegalArgumentException("Incorrect operand type " + o);
+ }
+
+ int returnValue = 0;
+
+ switch (getReportType()) {
+ case CONGESTION:
+ returnValue = ranking.get(getRuntimeType()) - ranking.get(other.getRuntimeType());
+ break;
+
+ case THROUGHPUT:
+ returnValue = (int) (other.getValue() - getValue());
+ break;
+ }
+
+ return returnValue;
+ }
+
+ private static Map<FeedRuntimeType, Integer> ranking = populateRanking();
+
+ private static Map<FeedRuntimeType, Integer> populateRanking() {
+ Map<FeedRuntimeType, Integer> ranking = new HashMap<FeedRuntimeType, Integer>();
+ ranking.put(FeedRuntimeType.INGESTION, 1);
+ ranking.put(FeedRuntimeType.COMPUTE, 2);
+ ranking.put(FeedRuntimeType.STORAGE, 3);
+ ranking.put(FeedRuntimeType.COMMIT, 4);
+ return ranking;
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedRuntime.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedRuntime.java
new file mode 100644
index 0000000..af25be2
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedRuntime.java
@@ -0,0 +1,133 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+
+public class FeedRuntime {
+
+ public enum FeedRuntimeType {
+ INGESTION,
+ COMPUTE,
+ STORAGE,
+ COMMIT
+ }
+
+ protected final FeedRuntimeId feedRuntimeId;
+
+ protected FeedRuntimeState runtimeState;
+
+ public FeedRuntime(FeedConnectionId feedId, int partition, FeedRuntimeType feedRuntimeType) {
+ this.feedRuntimeId = new FeedRuntimeId(feedRuntimeType, feedId, partition);
+ }
+
+ public FeedRuntime(FeedConnectionId feedId, int partition, FeedRuntimeType feedRuntimeType,
+ FeedRuntimeState runtimeState) {
+ this.feedRuntimeId = new FeedRuntimeId(feedRuntimeType, feedId, partition);
+ this.runtimeState = runtimeState;
+ }
+
+ @Override
+ public String toString() {
+ return feedRuntimeId + " " + "runtime state ? " + (runtimeState != null);
+ }
+
+ public static class FeedRuntimeState {
+
+ private ByteBuffer frame;
+ private IFrameWriter frameWriter;
+ private Exception exception;
+
+ public FeedRuntimeState(ByteBuffer frame, IFrameWriter frameWriter, Exception e) {
+ this.frame = frame;
+ this.frameWriter = frameWriter;
+ this.exception = exception;
+ }
+
+ public ByteBuffer getFrame() {
+ return frame;
+ }
+
+ public void setFrame(ByteBuffer frame) {
+ this.frame = frame;
+ }
+
+ public IFrameWriter getFrameWriter() {
+ return frameWriter;
+ }
+
+ public void setFrameWriter(IFrameWriter frameWriter) {
+ this.frameWriter = frameWriter;
+ }
+
+ public Exception getException() {
+ return exception;
+ }
+
+ public void setException(Exception exception) {
+ this.exception = exception;
+ }
+
+ }
+
+ public static class FeedRuntimeId {
+
+ private final FeedRuntimeType feedRuntimeType;
+ private final FeedConnectionId feedId;
+ private final int partition;
+ private final int hashCode;
+
+ public FeedRuntimeId(FeedRuntimeType runtimeType, FeedConnectionId feedId, int partition) {
+ this.feedRuntimeType = runtimeType;
+ this.feedId = feedId;
+ this.partition = partition;
+ this.hashCode = (feedId + "[" + partition + "]" + feedRuntimeType).hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return feedId + "[" + partition + "]" + " " + feedRuntimeType;
+ }
+
+ @Override
+ public int hashCode() {
+ return hashCode;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o instanceof FeedRuntimeId) {
+ FeedRuntimeId oid = ((FeedRuntimeId) o);
+ return oid.getFeedId().equals(feedId) && oid.getFeedRuntimeType().equals(feedRuntimeType)
+ && oid.getPartition() == partition;
+ }
+ return false;
+ }
+
+ public FeedRuntimeType getFeedRuntimeType() {
+ return feedRuntimeType;
+ }
+
+ public FeedConnectionId getFeedId() {
+ return feedId;
+ }
+
+ public int getPartition() {
+ return partition;
+ }
+
+ }
+
+ public FeedRuntimeState getRuntimeState() {
+ return runtimeState;
+ }
+
+ public void setRuntimeState(FeedRuntimeState runtimeState) {
+ this.runtimeState = runtimeState;
+ }
+
+ public FeedRuntimeId getFeedRuntimeId() {
+ return feedRuntimeId;
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedRuntimeManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedRuntimeManager.java
new file mode 100644
index 0000000..b4951ec
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedRuntimeManager.java
@@ -0,0 +1,234 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.IOException;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeId;
+
+public class FeedRuntimeManager {
+
+ private static Logger LOGGER = Logger.getLogger(FeedRuntimeManager.class.getName());
+
+ private final FeedConnectionId feedId;
+ private SuperFeedManager superFeedManager;
+ private final Map<FeedRuntimeId, FeedRuntime> feedRuntimes;
+ private final ExecutorService executorService;
+ private FeedMessageService messageService;
+ private SocketFactory socketFactory = new SocketFactory();
+ private final LinkedBlockingQueue<String> feedReportQueue;
+
+ public FeedRuntimeManager(FeedConnectionId feedId) {
+ this.feedId = feedId;
+ feedRuntimes = new ConcurrentHashMap<FeedRuntimeId, FeedRuntime>();
+ executorService = Executors.newCachedThreadPool();
+ feedReportQueue = new LinkedBlockingQueue<String>();
+ }
+
+ public void close(boolean closeAll) throws IOException {
+ socketFactory.close();
+
+ if (messageService != null) {
+ messageService.stop();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Shut down message service s for :" + feedId);
+ }
+ messageService = null;
+ }
+ if (superFeedManager != null && superFeedManager.isLocal()) {
+ superFeedManager.stop();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Shut down super feed manager for :" + feedId);
+ }
+ }
+
+ if (closeAll) {
+ if (executorService != null) {
+ executorService.shutdownNow();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Shut down executor service for :" + feedId);
+ }
+ }
+ }
+ }
+
+ public void setSuperFeedManager(SuperFeedManager sfm) throws UnknownHostException, IOException, Exception {
+ this.superFeedManager = sfm;
+ if (sfm.isLocal()) {
+ sfm.start();
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Started Super Feed Manager for feed :" + feedId);
+ }
+ this.messageService = new FeedMessageService(feedId);
+ messageService.start();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Started Feed Message Service for feed :" + feedId);
+ }
+ }
+
+ public SuperFeedManager getSuperFeedManager() {
+ return superFeedManager;
+ }
+
+ public FeedRuntime getFeedRuntime(FeedRuntimeId runtimeId) {
+ return feedRuntimes.get(runtimeId);
+ }
+
+ public void registerFeedRuntime(FeedRuntimeId runtimeId, FeedRuntime feedRuntime) {
+ feedRuntimes.put(runtimeId, feedRuntime);
+ }
+
+ public void deregisterFeedRuntime(FeedRuntimeId runtimeId) {
+ feedRuntimes.remove(runtimeId);
+ if (feedRuntimes.isEmpty()) {
+ synchronized (this) {
+ if (feedRuntimes.isEmpty()) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("De-registering feed");
+ }
+ FeedManager.INSTANCE.deregisterFeed(runtimeId.getFeedId());
+ }
+ }
+ }
+ }
+
+ public ExecutorService getExecutorService() {
+ return executorService;
+ }
+
+ public FeedMessageService getMessageService() {
+ return messageService;
+ }
+
+ public Socket createClientSocket(String host, int port) throws UnknownHostException, IOException {
+ return socketFactory.createClientSocket(host, port);
+ }
+
+ public Socket createClientSocket(String host, int port, long timeout) throws UnknownHostException, IOException {
+ Socket client = null;
+ boolean continueAttempt = true;
+ long startAttempt = System.currentTimeMillis();
+ long endAttempt = System.currentTimeMillis();
+ while (client == null && continueAttempt) {
+ try {
+ client = socketFactory.createClientSocket(host, port);
+ } catch (Exception e) {
+ endAttempt = System.currentTimeMillis();
+ if (endAttempt - startAttempt > timeout) {
+ continueAttempt = false;
+ }
+ }
+ }
+ return client;
+ }
+
+ public ServerSocket createServerSocket(int port) throws IOException {
+ return socketFactory.createServerSocket(port);
+ }
+
+ private static class SocketFactory {
+
+ private final Map<SocketId, Socket> sockets = new HashMap<SocketId, Socket>();
+ private final List<ServerSocket> serverSockets = new ArrayList<ServerSocket>();
+
+ public Socket createClientSocket(String host, int port) throws UnknownHostException, IOException {
+ Socket socket = new Socket(host, port);
+ sockets.put(new SocketId(host, port), socket);
+ return socket;
+ }
+
+ public void close() throws IOException {
+ for (ServerSocket socket : serverSockets) {
+ socket.close();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Closed server socket :" + socket);
+ }
+ }
+
+ for (Entry<SocketId, Socket> entry : sockets.entrySet()) {
+ entry.getValue().close();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Closed client socket :" + entry.getKey());
+ }
+ }
+ }
+
+ public ServerSocket createServerSocket(int port) throws IOException {
+ ServerSocket socket = new ServerSocket(port);
+ serverSockets.add(socket);
+ return socket;
+ }
+
+ public ServerSocket createServerSocket() throws IOException {
+ ServerSocket socket = new ServerSocket(0);
+ serverSockets.add(socket);
+ return socket;
+ }
+
+ private static class SocketId {
+ private final String host;
+ private final int port;
+
+ public SocketId(String host, int port) {
+ this.host = host;
+ this.port = port;
+ }
+
+ public SocketId(int port) {
+ this.host = "127.0.0.1";
+ this.port = port;
+ }
+
+ public String getHost() {
+ return host;
+ }
+
+ public int getPort() {
+ return port;
+ }
+
+ @Override
+ public String toString() {
+ return host + "[" + port + "]";
+ }
+
+ @Override
+ public int hashCode() {
+ return toString().hashCode();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (!(o instanceof SocketId)) {
+ return false;
+ }
+
+ return ((SocketId) o).getHost().equals(host) && ((SocketId) o).getPort() == port;
+ }
+
+ }
+ }
+
+ public FeedConnectionId getFeedId() {
+ return feedId;
+ }
+
+ public LinkedBlockingQueue<String> getFeedReportQueue() {
+ return feedReportQueue;
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedSourceProperties.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedSourceProperties.java
new file mode 100644
index 0000000..268e30a
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedSourceProperties.java
@@ -0,0 +1,46 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class FeedSourceProperties {
+
+ public enum ExchangeMode {
+ PULL,
+ PUSH
+ }
+
+ public enum IngestionRuntimeType {
+ STATELESS,
+ STATEFULL
+ }
+
+ public enum ComputeRuntimeType {
+ STATELESS,
+ STATEFULL
+ }
+
+ public static class FeedSourcePropertyKeys {
+ public static final String EXCHANGE_MODE = "exchange_mode";
+ public static final String INGESTION_RUNTIME_TYPE = "ingestion_runtime_type";
+ public static final String COMPUTE_RUNTIME_TYPE = "compute_runtime_type";
+ public static final String BACKWARD_TIME_TRAVEL = "backward_time_travel";
+ public static final String COMPUTE_IDEMPOTENCE = "compute_idempotence";
+ }
+
+ private Map<String, String> sourceConfiguration;
+
+ public FeedSourceProperties(Map<String, String> sourceConfiguration) {
+ this.sourceConfiguration = sourceConfiguration;
+ }
+
+ public static class FeedSourcePropertyAccessor {
+
+ private boolean computeIdempotence;
+ private boolean backwardTimeTravel;
+ private ExchangeMode exchangeMode;
+ private IngestionRuntimeType ingestionRuntimeType;
+ private ComputeRuntimeType computeRuntimeType;
+
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedUtil.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedUtil.java
new file mode 100644
index 0000000..c1cddb7
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedUtil.java
@@ -0,0 +1,229 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.commons.lang3.tuple.Pair;
+
+import edu.uci.ics.asterix.common.dataflow.AsterixLSMTreeInsertDeleteOperatorDescriptor;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
+import edu.uci.ics.asterix.metadata.entities.Feed;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityType;
+import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.std.StreamProjectRuntimeFactory;
+import edu.uci.ics.hyracks.api.constraints.Constraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.LValueConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.PartitionCountExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.PartitionLocationExpression;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class FeedUtil {
+
+ private static Logger LOGGER = Logger.getLogger(FeedUtil.class.getName());
+
+ public static boolean isFeedActive(FeedActivity feedActivity) {
+ return (feedActivity != null && !(feedActivity.getActivityType().equals(FeedActivityType.FEED_FAILURE) || feedActivity
+ .getActivityType().equals(FeedActivityType.FEED_END)));
+ }
+
+ public static JobSpecification alterJobSpecificationForFeed(JobSpecification spec,
+ FeedConnectionId feedConnectionId, FeedPolicy feedPolicy) {
+ JobSpecification altered = null;
+ altered = new JobSpecification();
+ Map<OperatorDescriptorId, IOperatorDescriptor> operatorMap = spec.getOperatorMap();
+
+ // copy operators
+ Map<OperatorDescriptorId, OperatorDescriptorId> oldNewOID = new HashMap<OperatorDescriptorId, OperatorDescriptorId>();
+ for (Entry<OperatorDescriptorId, IOperatorDescriptor> entry : operatorMap.entrySet()) {
+ IOperatorDescriptor opDesc = entry.getValue();
+ if (opDesc instanceof FeedIntakeOperatorDescriptor) {
+ FeedIntakeOperatorDescriptor orig = (FeedIntakeOperatorDescriptor) opDesc;
+ FeedIntakeOperatorDescriptor fiop = new FeedIntakeOperatorDescriptor(altered, orig.getFeedId(),
+ orig.getAdapterFactory(), (ARecordType) orig.getAtype(), orig.getRecordDescriptor(),
+ orig.getFeedPolicy());
+ oldNewOID.put(opDesc.getOperatorId(), fiop.getOperatorId());
+ } else if (opDesc instanceof AsterixLSMTreeInsertDeleteOperatorDescriptor) {
+ FeedMetaOperatorDescriptor metaOp = new FeedMetaOperatorDescriptor(altered, feedConnectionId, opDesc,
+ feedPolicy, FeedRuntimeType.STORAGE);
+ oldNewOID.put(opDesc.getOperatorId(), metaOp.getOperatorId());
+ } else {
+ FeedRuntimeType runtimeType = null;
+ if (opDesc instanceof AlgebricksMetaOperatorDescriptor) {
+ IPushRuntimeFactory runtimeFactory = ((AlgebricksMetaOperatorDescriptor) opDesc).getPipeline()
+ .getRuntimeFactories()[0];
+ if (runtimeFactory instanceof AssignRuntimeFactory) {
+ runtimeType = FeedRuntimeType.COMPUTE;
+ } else if (runtimeFactory instanceof StreamProjectRuntimeFactory) {
+ runtimeType = FeedRuntimeType.COMMIT;
+ }
+ }
+ FeedMetaOperatorDescriptor metaOp = new FeedMetaOperatorDescriptor(altered, feedConnectionId, opDesc,
+ feedPolicy, runtimeType);
+
+ oldNewOID.put(opDesc.getOperatorId(), metaOp.getOperatorId());
+ }
+ }
+
+ // copy connectors
+ Map<ConnectorDescriptorId, ConnectorDescriptorId> connectorMapping = new HashMap<ConnectorDescriptorId, ConnectorDescriptorId>();
+ for (Entry<ConnectorDescriptorId, IConnectorDescriptor> entry : spec.getConnectorMap().entrySet()) {
+ IConnectorDescriptor connDesc = entry.getValue();
+ ConnectorDescriptorId newConnId = altered.createConnectorDescriptor(connDesc);
+ connectorMapping.put(entry.getKey(), newConnId);
+ }
+
+ // make connections between operators
+ for (Entry<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>> entry : spec
+ .getConnectorOperatorMap().entrySet()) {
+ IConnectorDescriptor connDesc = altered.getConnectorMap().get(connectorMapping.get(entry.getKey()));
+ Pair<IOperatorDescriptor, Integer> leftOp = entry.getValue().getLeft();
+ Pair<IOperatorDescriptor, Integer> rightOp = entry.getValue().getRight();
+
+ IOperatorDescriptor leftOpDesc = altered.getOperatorMap().get(
+ oldNewOID.get(leftOp.getLeft().getOperatorId()));
+ IOperatorDescriptor rightOpDesc = altered.getOperatorMap().get(
+ oldNewOID.get(rightOp.getLeft().getOperatorId()));
+
+ altered.connect(connDesc, leftOpDesc, leftOp.getRight(), rightOpDesc, rightOp.getRight());
+ }
+
+ // prepare for setting partition constraints
+ Map<OperatorDescriptorId, Map<Integer, String>> operatorLocations = new HashMap<OperatorDescriptorId, Map<Integer, String>>();
+ Map<OperatorDescriptorId, Integer> operatorCounts = new HashMap<OperatorDescriptorId, Integer>();
+
+ for (Constraint constraint : spec.getUserConstraints()) {
+ LValueConstraintExpression lexpr = constraint.getLValue();
+ ConstraintExpression cexpr = constraint.getRValue();
+ OperatorDescriptorId opId;
+ switch (lexpr.getTag()) {
+ case PARTITION_COUNT:
+ opId = ((PartitionCountExpression) lexpr).getOperatorDescriptorId();
+ operatorCounts.put(opId, (int) ((ConstantExpression) cexpr).getValue());
+ break;
+ case PARTITION_LOCATION:
+ opId = ((PartitionLocationExpression) lexpr).getOperatorDescriptorId();
+ IOperatorDescriptor opDesc = altered.getOperatorMap().get(oldNewOID.get(opId));
+ Map<Integer, String> locations = operatorLocations.get(opDesc.getOperatorId());
+ if (locations == null) {
+ locations = new HashMap<Integer, String>();
+ operatorLocations.put(opDesc.getOperatorId(), locations);
+ }
+ String location = (String) ((ConstantExpression) cexpr).getValue();
+ int partition = ((PartitionLocationExpression) lexpr).getPartition();
+ locations.put(partition, location);
+ break;
+ }
+ }
+
+ // set absolute location constraints
+ for (Entry<OperatorDescriptorId, Map<Integer, String>> entry : operatorLocations.entrySet()) {
+ IOperatorDescriptor opDesc = altered.getOperatorMap().get(oldNewOID.get(entry.getKey()));
+ String[] locations = new String[entry.getValue().size()];
+ for (Entry<Integer, String> e : entry.getValue().entrySet()) {
+ locations[e.getKey()] = e.getValue();
+ }
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(altered, opDesc, locations);
+ }
+
+ // set count constraints
+ for (Entry<OperatorDescriptorId, Integer> entry : operatorCounts.entrySet()) {
+ IOperatorDescriptor opDesc = altered.getOperatorMap().get(oldNewOID.get(entry.getKey()));
+ if (!operatorLocations.keySet().contains(entry.getKey())) {
+ PartitionConstraintHelper.addPartitionCountConstraint(altered, opDesc, entry.getValue());
+ }
+ }
+
+ // useConnectorSchedulingPolicy
+ altered.setUseConnectorPolicyForScheduling(spec.isUseConnectorPolicyForScheduling());
+
+ // connectorAssignmentPolicy
+ altered.setConnectorPolicyAssignmentPolicy(spec.getConnectorPolicyAssignmentPolicy());
+
+ // roots
+ for (OperatorDescriptorId root : spec.getRoots()) {
+ altered.addRoot(altered.getOperatorMap().get(oldNewOID.get(root)));
+ }
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("New Job Spec:" + altered);
+ }
+
+ return altered;
+
+ }
+
+ public static Pair<IAdapterFactory, ARecordType> getFeedFactoryAndOutput(Feed feed,
+ MetadataTransactionContext mdTxnCtx) throws AlgebricksException {
+
+ String adapterName = null;
+ DatasourceAdapter adapterEntity = null;
+ String adapterFactoryClassname = null;
+ IAdapterFactory adapterFactory = null;
+ ARecordType adapterOutputType = null;
+ Pair<IAdapterFactory, ARecordType> feedProps = null;
+ try {
+ adapterName = feed.getAdaptorName();
+ adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
+ adapterName);
+ if (adapterEntity != null) {
+ adapterFactoryClassname = adapterEntity.getClassname();
+ adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+ } else {
+ adapterFactoryClassname = AqlMetadataProvider.adapterFactoryMapping.get(adapterName);
+ if (adapterFactoryClassname != null) {
+ } else {
+ adapterFactoryClassname = adapterName;
+ }
+ adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+ }
+
+ Map<String, String> configuration = feed.getAdaptorConfiguration();
+
+ switch (adapterFactory.getAdapterType()) {
+ case TYPED:
+ adapterOutputType = ((ITypedAdapterFactory) adapterFactory).getAdapterOutputType();
+ ((ITypedAdapterFactory) adapterFactory).configure(configuration);
+ break;
+ case GENERIC:
+ String outputTypeName = configuration.get("output-type-name");
+ adapterOutputType = (ARecordType) MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
+ feed.getDataverseName(), outputTypeName).getDatatype();
+ ((IGenericAdapterFactory) adapterFactory).configure(configuration, (ARecordType) adapterOutputType);
+ break;
+ default:
+ throw new IllegalStateException(" Unknown factory type for " + adapterFactoryClassname);
+ }
+
+ feedProps = Pair.of(adapterFactory, adapterOutputType);
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new AlgebricksException("unable to create adapter " + e);
+ }
+ return feedProps;
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterExecutor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterExecutor.java
new file mode 100644
index 0000000..0d1f955
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterExecutor.java
@@ -0,0 +1,20 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+public interface IAdapterExecutor {
+
+ /**
+ * @throws Exception
+ */
+ public void start() throws Exception;
+
+ /**
+ * @throws Exception
+ */
+ public void stop() throws Exception;
+
+ /**
+ * @return
+ */
+ public FeedConnectionId getFeedId();
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterFactory.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterFactory.java
new file mode 100644
index 0000000..da4f131
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterFactory.java
@@ -0,0 +1,90 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * Base interface for IGenericDatasetAdapterFactory and ITypedDatasetAdapterFactory.
+ * Acts as a marker interface indicating that the implementation provides functionality
+ * for creating an adapter.
+ */
+public interface IAdapterFactory extends Serializable {
+
+ /**
+ * A 'GENERIC' adapter can be configured to return a given datatype.
+ * A 'TYPED' adapter returns records with a pre-defined datatype.
+ */
+ public enum AdapterType {
+ GENERIC,
+ TYPED
+ }
+
+ public enum SupportedOperation {
+ READ,
+ WRITE,
+ READ_WRITE
+ }
+
+ /**
+ * Returns the type of adapter indicating if the adapter can be used for
+ * reading from an external data source or writing to an external data
+ * source or can be used for both purposes.
+ *
+ * @see SupportedOperation
+ * @return
+ */
+ public SupportedOperation getSupportedOperations();
+
+ /**
+ * Returns the display name corresponding to the Adapter type that is created by the factory.
+ *
+ * @return the display name
+ */
+ public String getName();
+
+ /**
+ * Returns the type of the adapter (GENERIC or TYPED)
+ *
+ * @return
+ */
+ public AdapterType getAdapterType();
+
+ /**
+ * Returns a list of partition constraints. A partition constraint can be a
+ * requirement to execute at a particular location or could be cardinality
+ * constraints indicating the number of instances that need to run in
+ * parallel. example, a IDatasourceAdapter implementation written for data
+ * residing on the local file system of a node cannot run on any other node
+ * and thus has a location partition constraint. The location partition
+ * constraint can be expressed as a node IP address or a node controller id.
+ * In the former case, the IP address is translated to a node controller id
+ * running on the node with the given IP address.
+ */
+ public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception;
+
+ /**
+ * Creates an instance of IDatasourceAdapter.
+ *
+ * @param HyracksTaskContext
+ * @return An instance of IDatasourceAdapter.
+ * @throws Exception
+ */
+ public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception;
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IDatasourceAdapter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IDatasourceAdapter.java
new file mode 100644
index 0000000..d0f8cdd
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IDatasourceAdapter.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+
+/**
+ * A super interface implemented by a data source adapter. An adapter can be a
+ * pull based or push based. This interface provides all common APIs that need
+ * to be implemented by each adapter irrespective of the the kind of
+ * adapter(pull or push).
+ */
+public interface IDatasourceAdapter extends Serializable {
+
+ /**
+ * An adapter can be used to read from an external data source and may also
+ * allow writing to the external data source. This enum type indicates the
+ * kind of operations supported by the adapter.
+ *
+ * @caller Compiler uses this method to assert the validity of an operation
+ * on an external dataset. The type of adapter associated with an
+ * external dataset determines the set of valid operations allowed
+ * on the dataset.
+ */
+
+
+ /**
+ * Triggers the adapter to begin ingesting data from the external source.
+ *
+ * @param partition
+ * The adapter could be running with a degree of parallelism.
+ * partition corresponds to the i'th parallel instance.
+ * @param writer
+ * The instance of frame writer that is used by the adapter to
+ * write frame to. Adapter packs the fetched bytes (from external source),
+ * packs them into frames and forwards the frames to an upstream receiving
+ * operator using the instance of IFrameWriter.
+ * @throws Exception
+ */
+ public void start(int partition, IFrameWriter writer) throws Exception;
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/ITypedDatasourceAdapter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedAdapter.java
similarity index 63%
copy from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/ITypedDatasourceAdapter.java
copy to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedAdapter.java
index 00b8530..f525420 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/ITypedDatasourceAdapter.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedAdapter.java
@@ -12,16 +12,19 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.external.dataset.adapter;
-
-import edu.uci.ics.asterix.om.types.ARecordType;
+package edu.uci.ics.asterix.metadata.feeds;
/**
- * Implemented by datasource adapter that has a fixed output type.
- * Example @see {PullBasedTwitterAdapter}
+ * Interface implemented by an adapter that can be controlled or managed by external
+ * commands (stop,alter)
*/
-public interface ITypedDatasourceAdapter extends IDatasourceAdapter {
+public interface IFeedAdapter extends IDatasourceAdapter {
- public ARecordType getAdapterOutputType();
+ /**
+ * Discontinue the ingestion of data and end the feed.
+ *
+ * @throws Exception
+ */
+ public void stop() throws Exception;
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedManager.java
new file mode 100644
index 0000000..ac33966
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedManager.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeId;
+
+/**
+ * Handle (de)registration of feeds for delivery of control messages.
+ */
+public interface IFeedManager {
+
+ /**
+ * @param feedId
+ * @return
+ */
+ public ExecutorService getFeedExecutorService(FeedConnectionId feedId);
+
+ /**
+ * @param feedRuntime
+ * @throws Exception
+ */
+ public void registerFeedRuntime(FeedRuntime feedRuntime) throws Exception;
+
+ /**
+ * @param feedRuntimeId
+ */
+ public void deRegisterFeedRuntime(FeedRuntimeId feedRuntimeId);
+
+ /**
+ * @param feedRuntimeId
+ * @return
+ */
+ public FeedRuntime getFeedRuntime(FeedRuntimeId feedRuntimeId);
+
+ /**
+ * @param feedId
+ * @param sfm
+ * @throws Exception
+ */
+ public void registerSuperFeedManager(FeedConnectionId feedId, SuperFeedManager sfm) throws Exception;
+
+ /**
+ * @param feedId
+ * @return
+ */
+ public SuperFeedManager getSuperFeedManager(FeedConnectionId feedId);
+
+ /**
+ * @param feedId
+ * @throws IOException
+ */
+ void deregisterFeed(FeedConnectionId feedId) throws IOException;
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedMessage.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedMessage.java
similarity index 89%
rename from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedMessage.java
rename to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedMessage.java
index dcef2c8..0241e5b 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/feed/lifecycle/IFeedMessage.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedMessage.java
@@ -12,15 +12,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.external.feed.lifecycle;
+package edu.uci.ics.asterix.metadata.feeds;
import java.io.Serializable;
public interface IFeedMessage extends Serializable {
public enum MessageType {
- STOP,
- ALTER,
+ END,
+ SUPER_FEED_MANAGER_ELECT
}
public MessageType getMessageType();
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IGenericAdapterFactory.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IGenericAdapterFactory.java
new file mode 100644
index 0000000..ac0c711
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IGenericAdapterFactory.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.om.types.ARecordType;
+
+public interface IGenericAdapterFactory extends IAdapterFactory {
+
+ public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception;
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/ITypedDatasourceAdapter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ITypedAdapterFactory.java
similarity index 74%
rename from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/ITypedDatasourceAdapter.java
rename to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ITypedAdapterFactory.java
index 00b8530..6faa44b 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/ITypedDatasourceAdapter.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ITypedAdapterFactory.java
@@ -12,16 +12,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.external.dataset.adapter;
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.Map;
import edu.uci.ics.asterix.om.types.ARecordType;
-/**
- * Implemented by datasource adapter that has a fixed output type.
- * Example @see {PullBasedTwitterAdapter}
- */
-public interface ITypedDatasourceAdapter extends IDatasourceAdapter {
+public interface ITypedAdapterFactory extends IAdapterFactory {
public ARecordType getAdapterOutputType();
+ public void configure(Map<String, String> configuration) throws Exception;
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IngestionRuntime.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IngestionRuntime.java
new file mode 100644
index 0000000..37d49b4
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IngestionRuntime.java
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.feeds;
+
+public class IngestionRuntime extends FeedRuntime {
+
+ private AdapterRuntimeManager adapterRuntimeManager;
+
+ public IngestionRuntime(FeedConnectionId feedId, int partition, FeedRuntimeType feedRuntimeType,
+ AdapterRuntimeManager adaptorRuntimeManager) {
+ super(feedId, partition, feedRuntimeType);
+ this.adapterRuntimeManager = adaptorRuntimeManager;
+ }
+
+ public AdapterRuntimeManager getAdapterRuntimeManager() {
+ return adapterRuntimeManager;
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/MessageListener.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/MessageListener.java
new file mode 100644
index 0000000..20a6b0e
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/MessageListener.java
@@ -0,0 +1,165 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.nio.CharBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public class MessageListener {
+
+ private static final Logger LOGGER = Logger.getLogger(MessageListener.class.getName());
+
+ private int port;
+ private final LinkedBlockingQueue<String> outbox;
+
+ private ExecutorService executorService = Executors.newFixedThreadPool(10);
+
+ private MessageListenerServer listenerServer;
+
+ public MessageListener(int port, LinkedBlockingQueue<String> outbox) {
+ this.port = port;
+ this.outbox = outbox;
+ }
+
+ public void stop() {
+ listenerServer.stop();
+ System.out.println("STOPPED MESSAGE RECEIVING SERVICE AT " + port);
+ if (!executorService.isShutdown()) {
+ executorService.shutdownNow();
+ }
+
+ }
+
+ public void start() throws IOException {
+ System.out.println("STARTING MESSAGE RECEIVING SERVICE AT " + port);
+ listenerServer = new MessageListenerServer(port, outbox);
+ executorService.execute(listenerServer);
+ }
+
+ private static class MessageListenerServer implements Runnable {
+
+ private final int port;
+ private final LinkedBlockingQueue<String> outbox;
+ private ServerSocket server;
+
+ public MessageListenerServer(int port, LinkedBlockingQueue<String> outbox) {
+ this.port = port;
+ this.outbox = outbox;
+ }
+
+ public void stop() {
+ try {
+ server.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+
+ @Override
+ public void run() {
+ char EOL = (char) "\n".getBytes()[0];
+ Socket client = null;
+ try {
+ server = new ServerSocket(port);
+ client = server.accept();
+ InputStream in = client.getInputStream();
+ CharBuffer buffer = CharBuffer.allocate(5000);
+ char ch;
+ while (true) {
+ ch = (char) in.read();
+ if (((int) ch) == -1) {
+ break;
+ }
+ while (ch != EOL) {
+ buffer.put(ch);
+ ch = (char) in.read();
+ }
+ buffer.flip();
+ String s = new String(buffer.array());
+ synchronized (outbox) {
+ outbox.add(s + "\n");
+ }
+ buffer.position(0);
+ buffer.limit(5000);
+ }
+
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to start Message listener" + server);
+ }
+ } finally {
+ if (server != null) {
+ try {
+ server.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
+ }
+
+ }
+
+ private static class MessageParser implements Runnable {
+
+ private Socket client;
+ private IMessageAnalyzer messageAnalyzer;
+ private static final char EOL = (char) "\n".getBytes()[0];
+
+ public MessageParser(Socket client, IMessageAnalyzer messageAnalyzer) {
+ this.client = client;
+ this.messageAnalyzer = messageAnalyzer;
+ }
+
+ @Override
+ public void run() {
+ CharBuffer buffer = CharBuffer.allocate(5000);
+ char ch;
+ try {
+ InputStream in = client.getInputStream();
+ while (true) {
+ ch = (char) in.read();
+ if (((int) ch) == -1) {
+ break;
+ }
+ while (ch != EOL) {
+ buffer.put(ch);
+ ch = (char) in.read();
+ }
+ buffer.flip();
+ String s = new String(buffer.array());
+ synchronized (messageAnalyzer) {
+ messageAnalyzer.getMessageQueue().add(s + "\n");
+ }
+ buffer.position(0);
+ buffer.limit(5000);
+ }
+ } catch (IOException ioe) {
+ ioe.printStackTrace();
+ } finally {
+ try {
+ client.close();
+ } catch (IOException ioe) {
+ // do nothing
+ }
+ }
+ }
+ }
+
+ public static interface IMessageAnalyzer {
+
+ /**
+ * @return
+ */
+ public LinkedBlockingQueue<String> getMessageQueue();
+
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/RemoteSocketMessageListener.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/RemoteSocketMessageListener.java
new file mode 100644
index 0000000..822c638
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/RemoteSocketMessageListener.java
@@ -0,0 +1,165 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.Socket;
+import java.nio.CharBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public class RemoteSocketMessageListener {
+
+ private static final Logger LOGGER = Logger.getLogger(RemoteSocketMessageListener.class.getName());
+
+ private final String host;
+ private final int port;
+ private final LinkedBlockingQueue<String> outbox;
+
+ private ExecutorService executorService = Executors.newFixedThreadPool(10);
+
+ private RemoteMessageListenerServer listenerServer;
+
+ public RemoteSocketMessageListener(String host, int port, LinkedBlockingQueue<String> outbox) {
+ this.host = host;
+ this.port = port;
+ this.outbox = outbox;
+ }
+
+ public void stop() {
+ if (!executorService.isShutdown()) {
+ executorService.shutdownNow();
+ }
+ listenerServer.stop();
+
+ }
+
+ public void start() throws IOException {
+ listenerServer = new RemoteMessageListenerServer(host, port, outbox);
+ executorService.execute(listenerServer);
+ }
+
+ private static class RemoteMessageListenerServer implements Runnable {
+
+ private final String host;
+ private final int port;
+ private final LinkedBlockingQueue<String> outbox;
+ private Socket client;
+
+ public RemoteMessageListenerServer(String host, int port, LinkedBlockingQueue<String> outbox) {
+ this.host = host;
+ this.port = port;
+ this.outbox = outbox;
+ }
+
+ public void stop() {
+ try {
+ client.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+
+ @Override
+ public void run() {
+ char EOL = (char) "\n".getBytes()[0];
+ Socket client = null;
+ try {
+ client = new Socket(host, port);
+ InputStream in = client.getInputStream();
+ CharBuffer buffer = CharBuffer.allocate(5000);
+ char ch;
+ while (true) {
+ ch = (char) in.read();
+ if (((int) ch) == -1) {
+ break;
+ }
+ while (ch != EOL) {
+ buffer.put(ch);
+ ch = (char) in.read();
+ }
+ buffer.flip();
+ String s = new String(buffer.array());
+ synchronized (outbox) {
+ outbox.add(s + "\n");
+ }
+ buffer.position(0);
+ buffer.limit(5000);
+ }
+
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to start Remote Message listener" + client);
+ }
+ } finally {
+ if (client != null && !client.isClosed()) {
+ try {
+ client.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
+ }
+
+ }
+
+ private static class MessageParser implements Runnable {
+
+ private Socket client;
+ private IMessageAnalyzer messageAnalyzer;
+ private static final char EOL = (char) "\n".getBytes()[0];
+
+ public MessageParser(Socket client, IMessageAnalyzer messageAnalyzer) {
+ this.client = client;
+ this.messageAnalyzer = messageAnalyzer;
+ }
+
+ @Override
+ public void run() {
+ CharBuffer buffer = CharBuffer.allocate(5000);
+ char ch;
+ try {
+ InputStream in = client.getInputStream();
+ while (true) {
+ ch = (char) in.read();
+ if (((int) ch) == -1) {
+ break;
+ }
+ while (ch != EOL) {
+ buffer.put(ch);
+ ch = (char) in.read();
+ }
+ buffer.flip();
+ String s = new String(buffer.array());
+ synchronized (messageAnalyzer) {
+ messageAnalyzer.getMessageQueue().add(s + "\n");
+ }
+ buffer.position(0);
+ buffer.limit(5000);
+ }
+ } catch (IOException ioe) {
+ ioe.printStackTrace();
+ } finally {
+ try {
+ client.close();
+ } catch (IOException ioe) {
+ // do nothing
+ }
+ }
+ }
+ }
+
+ public static interface IMessageAnalyzer {
+
+ /**
+ * @return
+ */
+ public LinkedBlockingQueue<String> getMessageQueue();
+
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/SuperFeedManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/SuperFeedManager.java
new file mode 100644
index 0000000..a45456b6
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/SuperFeedManager.java
@@ -0,0 +1,443 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.metadata.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.metadata.feeds.MessageListener.IMessageAnalyzer;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
+
+/**
+ * The feed operators running in an NC report their health (statistics) to the local Feed Manager.
+ * A feed thus has a Feed Manager per NC. From amongst the Feed Maanger, a SuperFeedManager is chosen (randomly)
+ * The SuperFeedManager collects reports from the FeedMaangers and has the global cluster view in terms of
+ * how different feed operators running in a distributed fashion are performing.
+ */
+public class SuperFeedManager {
+
+ private static final Logger LOGGER = Logger.getLogger(SuperFeedManager.class.getName());
+
+ /**
+ * IP Address or DNS name of the host where Super Feed Manager is running.
+ */
+ private String host;
+
+ private AtomicInteger availablePort; // starting value is fixed
+
+ /**
+ * The port at which the SuperFeedManager listens for connections by other Feed Managers.
+ */
+ private final int feedReportPort; // fixed
+
+ /**
+ * The port at which the SuperFeedManager listens for connections by clients that wish
+ * to subscribe to the feed health reports.E.g. feed management console.
+ */
+ private final int feedReportSubscribePort; // fixed
+
+ /**
+ * The Id of Node Controller
+ */
+ private final String nodeId;
+
+ /**
+ * A unique identifier for the feed instance. A feed instance represents the flow of data
+ * from a feed to a dataset.
+ **/
+ private final FeedConnectionId feedConnectionId;
+
+ /**
+ * Set to true of the Super Feed Manager is local to the NC.
+ **/
+ private boolean isLocal = false;
+
+ private FeedReportDestinationSocketProvider sfmService;
+
+ private SuperFeedReportSubscriptionService subscriptionService;
+
+ private LinkedBlockingQueue<String> feedReportInbox; ///
+
+ private boolean started = false;
+
+ public static final int PORT_RANGE_ASSIGNED = 10;
+
+ public enum FeedReportMessageType {
+ CONGESTION,
+ THROUGHPUT
+ }
+
+ public SuperFeedManager(FeedConnectionId feedId, String host, String nodeId, int port) throws Exception {
+ this.feedConnectionId = feedId;
+ this.nodeId = nodeId;
+ this.feedReportPort = port;
+ this.feedReportSubscribePort = port + 1;
+ this.availablePort = new AtomicInteger(feedReportSubscribePort + 1);
+ this.host = host;
+ this.feedReportInbox = new LinkedBlockingQueue<String>();
+ }
+
+ public int getPort() {
+ return feedReportPort;
+ }
+
+ public String getHost() throws Exception {
+ return host;
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ public FeedConnectionId getFeedConnectionId() {
+ return feedConnectionId;
+ }
+
+ public boolean isLocal() {
+ return isLocal;
+ }
+
+ public void setLocal(boolean isLocal) {
+ this.isLocal = isLocal;
+ }
+
+ public void start() throws IOException {
+ if (sfmService == null) {
+ ExecutorService executorService = FeedManager.INSTANCE.getFeedExecutorService(feedConnectionId);
+ sfmService = new FeedReportDestinationSocketProvider(feedReportPort, feedReportInbox, feedConnectionId,
+ availablePort);
+ executorService.execute(sfmService);
+ subscriptionService = new SuperFeedReportSubscriptionService(feedConnectionId, feedReportSubscribePort,
+ sfmService.getMesgAnalyzer(), availablePort);
+ executorService.execute(subscriptionService);
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Started super feed manager! " + this);
+ }
+ started = true;
+ }
+
+ public void stop() throws IOException {
+ sfmService.stop();
+ subscriptionService.stop();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Stopped super feed manager! " + this);
+ }
+ started = false;
+ }
+
+ public boolean isStarted() {
+ return started;
+ }
+
+ @Override
+ public String toString() {
+ return feedConnectionId + "[" + nodeId + "(" + host + ")" + ":" + feedReportPort + "]"
+ + (isLocal ? started ? "Started " : "Not Started" : " Remote ");
+ }
+
+ public AtomicInteger getAvailablePort() {
+ return availablePort;
+ }
+
+ private static class SuperFeedReportSubscriptionService implements Runnable {
+
+ private final FeedConnectionId feedId;
+ private ServerSocket serverFeedSubscribe;
+ private AtomicInteger subscriptionPort;
+ private boolean active = true;
+ private String EOM = "\n";
+ private final FeedReportProvider reportProvider;
+ private final List<FeedDataProviderService> dataProviders = new ArrayList<FeedDataProviderService>();
+
+ public SuperFeedReportSubscriptionService(FeedConnectionId feedId, int port, FeedReportProvider reportProvider,
+ AtomicInteger nextPort) throws IOException {
+ this.feedId = feedId;
+ serverFeedSubscribe = FeedManager.INSTANCE.getFeedRuntimeManager(feedId).createServerSocket(port);
+ this.subscriptionPort = nextPort;
+ this.reportProvider = reportProvider;
+ }
+
+ public void stop() {
+ active = false;
+ for (FeedDataProviderService dataProviderService : dataProviders) {
+ dataProviderService.stop();
+ }
+ }
+
+ @Override
+ public void run() {
+ while (active) {
+ try {
+ Socket client = serverFeedSubscribe.accept();
+ OutputStream os = client.getOutputStream();
+ int port = subscriptionPort.incrementAndGet();
+ LinkedBlockingQueue<String> reportInbox = new LinkedBlockingQueue<String>();
+ reportProvider.registerSubsription(reportInbox);
+ FeedDataProviderService dataProviderService = new FeedDataProviderService(feedId, port, reportInbox);
+ dataProviders.add(dataProviderService);
+ FeedManager.INSTANCE.getFeedRuntimeManager(feedId).getExecutorService()
+ .execute(dataProviderService);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Recevied subscription request for feed :" + feedId
+ + " Subscripton available at port " + subscriptionPort);
+ }
+ os.write((port + EOM).getBytes());
+ os.flush();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ }
+
+ private static class FeedDataProviderService implements Runnable {
+
+ private final FeedConnectionId feedId;
+ private int subscriptionPort;
+ private ServerSocket dataProviderSocket;
+ private LinkedBlockingQueue<String> inbox;
+ private boolean active = true;
+ private String EOM = "\n";
+
+ public FeedDataProviderService(FeedConnectionId feedId, int port, LinkedBlockingQueue<String> inbox)
+ throws IOException {
+ this.feedId = feedId;
+ this.subscriptionPort = port;
+ this.inbox = inbox;
+ dataProviderSocket = FeedManager.INSTANCE.getFeedRuntimeManager(feedId).createServerSocket(port);
+ }
+
+ @Override
+ public void run() {
+ try {
+ Socket client = dataProviderSocket.accept();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Subscriber to " + feedId + " data connected");
+ }
+ OutputStream os = client.getOutputStream();
+ while (active) {
+ String message = inbox.take();
+ os.write((message + EOM).getBytes());
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Unsubscribed from " + feedId + " disconnected");
+ }
+ } catch (IOException e) {
+ e.printStackTrace();
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+
+ public void stop() {
+ active = false;
+ }
+
+ @Override
+ public String toString() {
+ return "DATA_PROVIDER_" + feedId + "[" + subscriptionPort + "]";
+ }
+
+ }
+
+ private static class FeedReportDestinationSocketProvider implements Runnable {
+
+ private static final String EOM = "\n";
+
+ private AtomicInteger nextPort;
+ private final ServerSocket feedReportSocket;
+ private final LinkedBlockingQueue<String> inbox;
+ private final List<MessageListener> messageListeners;
+ private final FeedReportProvider mesgAnalyzer;
+ private final FeedConnectionId feedId;
+ private boolean process = true;
+
+ public FeedReportDestinationSocketProvider(int port, LinkedBlockingQueue<String> inbox,
+ FeedConnectionId feedId, AtomicInteger availablePort) throws IOException {
+ FeedRuntimeManager runtimeManager = FeedManager.INSTANCE.getFeedRuntimeManager(feedId);
+ this.feedReportSocket = runtimeManager.createServerSocket(port);
+ this.nextPort = availablePort;
+ this.inbox = inbox;
+ this.feedId = feedId;
+ this.messageListeners = new ArrayList<MessageListener>();
+ this.mesgAnalyzer = new FeedReportProvider(inbox, feedId);
+ FeedManager.INSTANCE.getFeedExecutorService(feedId).execute(mesgAnalyzer);
+ }
+
+ public void stop() {
+ process = false;
+ if (feedReportSocket != null) {
+ try {
+ feedReportSocket.close();
+ process = false;
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+ for (MessageListener listener : messageListeners) {
+ listener.stop();
+ }
+ mesgAnalyzer.stop();
+ }
+
+ @Override
+ public void run() {
+ Socket client = null;
+ while (process) {
+ try {
+ client = feedReportSocket.accept();
+ int port = nextPort.incrementAndGet();
+ /**
+ * MessageListener provides the functionality of listening at a port for messages
+ * and delivering each received message to an input queue (inbox).
+ */
+ MessageListener listener = new MessageListener(port, inbox);
+ listener.start();
+ synchronized (messageListeners) {
+ messageListeners.add(listener);
+ }
+ OutputStream os = client.getOutputStream();
+ os.write((port + EOM).getBytes());
+ os.flush();
+ } catch (IOException e) {
+ if (process == false) {
+ break;
+ }
+ } finally {
+ if (client != null) {
+ try {
+ client.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ }
+ }
+
+ public FeedReportProvider getMesgAnalyzer() {
+ return mesgAnalyzer;
+ }
+
+ }
+
+ /**
+ * The report messages sent by the feed operators are sent to the FeedReportProvider.
+ * FeedReportMessageAnalyzer is responsible for distributing the messages to the subscribers.
+ * The Feed Management Console is an example of a subscriber.
+ */
+ private static class FeedReportProvider implements Runnable {
+
+ private final LinkedBlockingQueue<String> inbox;
+ private final FeedConnectionId feedId;
+ private boolean process = true;
+ private final List<LinkedBlockingQueue<String>> subscriptionQueues;
+ private final Map<String, String> ingestionThroughputs;
+
+ public FeedReportProvider(LinkedBlockingQueue<String> inbox, FeedConnectionId feedId)
+ throws UnknownHostException, IOException {
+ this.inbox = inbox;
+ this.feedId = feedId;
+ this.subscriptionQueues = new ArrayList<LinkedBlockingQueue<String>>();
+ this.ingestionThroughputs = new HashMap<String, String>();
+ }
+
+ public void stop() {
+ process = false;
+ }
+
+ public void registerSubsription(LinkedBlockingQueue<String> subscriptionQueue) {
+ subscriptionQueues.add(subscriptionQueue);
+ }
+
+ public void deregisterSubsription(LinkedBlockingQueue<String> subscriptionQueue) {
+ subscriptionQueues.remove(subscriptionQueue);
+ }
+
+ public void run() {
+ StringBuilder finalMessage = new StringBuilder();
+ FeedReport report = new FeedReport();
+ while (process) {
+ try {
+ String message = inbox.take();
+ report.reset(message);
+ FeedReportMessageType mesgType = report.getReportType();
+ switch (mesgType) {
+ case THROUGHPUT:
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.warning("Feed Health Report " + message);
+ }
+ String[] msgComponents = message.split("\\|");
+ String partition = msgComponents[3];
+ String tput = msgComponents[4];
+ String timestamp = msgComponents[6];
+
+ boolean dispatchReport = true;
+ if (ingestionThroughputs.get(partition) == null) {
+ ingestionThroughputs.put(partition, tput);
+ dispatchReport = false;
+ } else {
+ for (int i = 0; i < ingestionThroughputs.size(); i++) {
+ String tp = ingestionThroughputs.get(i + "");
+ if (tp != null) {
+ ingestionThroughputs.put(i + "", null);
+ finalMessage.append(tp + "|");
+ } else {
+ dispatchReport = false;
+ break;
+ }
+ }
+ ingestionThroughputs.put(partition, tput);
+ }
+
+ if (dispatchReport) {
+ String dispatchedReport = finalMessage.toString();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Dispatched report " + dispatchedReport);
+ }
+ for (LinkedBlockingQueue<String> q : subscriptionQueues) {
+ q.add(dispatchedReport);
+ }
+ }
+ finalMessage.delete(0, finalMessage.length());
+ break;
+ case CONGESTION:
+ // congestionInbox.add(report);
+ break;
+ }
+ } catch (InterruptedException e) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Unable to process messages " + e.getMessage() + " for feed " + feedId);
+ }
+ }
+ }
+ }
+
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/AsterixExternalScalarFunctionInfo.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/AsterixExternalScalarFunctionInfo.java
new file mode 100644
index 0000000..6bf5be3
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/AsterixExternalScalarFunctionInfo.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.functions;
+
+import java.util.List;
+
+import edu.uci.ics.asterix.om.functions.AsterixExternalFunctionInfo;
+import edu.uci.ics.asterix.om.functions.AsterixFunction;
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind;
+
+public class AsterixExternalScalarFunctionInfo extends AsterixExternalFunctionInfo {
+
+ private static final long serialVersionUID = 1L;
+
+ public AsterixExternalScalarFunctionInfo(String namespace, AsterixFunction asterixFunction, IAType returnType,
+ String body, String language, List<IAType> argumentTypes, IResultTypeComputer rtc) {
+ super(namespace, asterixFunction, FunctionKind.SCALAR, argumentTypes, returnType, rtc, body, language);
+ }
+
+ public AsterixExternalScalarFunctionInfo() {
+ super();
+ }
+
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/ExternalFunctionCompilerUtil.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
new file mode 100644
index 0000000..f699c41
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
@@ -0,0 +1,221 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.metadata.functions;
+
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.entities.Datatype;
+import edu.uci.ics.asterix.metadata.entities.Function;
+import edu.uci.ics.asterix.om.functions.AsterixFunction;
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.ADoubleTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.AFloatTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.AInt32TypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.AStringTypeComputer;
+import edu.uci.ics.asterix.om.types.AOrderedListType;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+public class ExternalFunctionCompilerUtil implements Serializable {
+
+ private static Pattern orderedListPattern = Pattern.compile("\\[*\\]");
+ private static Pattern unorderedListPattern = Pattern.compile("[{{*}}]");
+
+
+ public static IFunctionInfo getExternalFunctionInfo(MetadataTransactionContext txnCtx, Function function)
+ throws MetadataException {
+
+ String functionKind = function.getKind();
+ IFunctionInfo finfo = null;
+ if (FunctionKind.SCALAR.toString().equalsIgnoreCase(functionKind)) {
+ finfo = getScalarFunctionInfo(txnCtx, function);
+ } else if (FunctionKind.AGGREGATE.toString().equalsIgnoreCase(functionKind)) {
+ finfo = getAggregateFunctionInfo(txnCtx, function);
+ } else if (FunctionKind.STATEFUL.toString().equalsIgnoreCase(functionKind)) {
+ finfo = getStatefulFunctionInfo(txnCtx, function);
+ } else if (FunctionKind.UNNEST.toString().equalsIgnoreCase(functionKind)) {
+ finfo = getUnnestFunctionInfo(txnCtx, function);
+ }
+ return finfo;
+ }
+
+ private static IFunctionInfo getScalarFunctionInfo(MetadataTransactionContext txnCtx, Function function)
+ throws MetadataException {
+ FunctionIdentifier fid = new FunctionIdentifier(function.getDataverseName(), function.getName(),
+ function.getArity());
+ IResultTypeComputer typeComputer = getResultTypeComputer(txnCtx, function);
+ List<IAType> arguments = new ArrayList<IAType>();
+ IAType returnType = null;
+ List<String> paramTypes = function.getParams();
+ for (String paramType : paramTypes) {
+ arguments.add(getTypeInfo(paramType, txnCtx, function));
+ }
+
+ returnType = getTypeInfo(function.getReturnType(), txnCtx, function);
+
+ AsterixExternalScalarFunctionInfo scalarFunctionInfo = new AsterixExternalScalarFunctionInfo(
+ fid.getNamespace(), new AsterixFunction(fid.getName(), fid.getArity()), returnType,
+ function.getFunctionBody(), function.getLanguage(), arguments, typeComputer);
+ return scalarFunctionInfo;
+ }
+
+ private static IAType getTypeInfo(String paramType, MetadataTransactionContext txnCtx, Function function)
+ throws MetadataException {
+ if (paramType.equalsIgnoreCase(BuiltinType.AINT32.getDisplayName())) {
+ return (BuiltinType.AINT32);
+ } else if (paramType.equalsIgnoreCase(BuiltinType.AFLOAT.getDisplayName())) {
+ return (BuiltinType.AFLOAT);
+ } else if (paramType.equalsIgnoreCase(BuiltinType.ASTRING.getDisplayName())) {
+ return (BuiltinType.ASTRING);
+ } else if (paramType.equalsIgnoreCase(BuiltinType.ADOUBLE.getDisplayName())) {
+ return (BuiltinType.ADOUBLE);
+ } else {
+ IAType collection = getCollectionType(paramType, txnCtx, function);
+ if (collection != null) {
+ return collection;
+ } else {
+ Datatype datatype;
+ datatype = MetadataManager.INSTANCE.getDatatype(txnCtx, function.getDataverseName(), paramType);
+ if (datatype == null) {
+ throw new MetadataException(" Type " + paramType + " not defined");
+ }
+ return (datatype.getDatatype());
+ }
+ }
+ }
+
+ private static IAType getCollectionType(String paramType, MetadataTransactionContext txnCtx, Function function)
+ throws MetadataException {
+
+ Matcher matcher = orderedListPattern.matcher(paramType);
+ if (matcher.find()) {
+ String subType = paramType.substring(paramType.indexOf('[') + 1, paramType.lastIndexOf(']'));
+ return new AOrderedListType(getTypeInfo(subType, txnCtx, function), "AOrderedList");
+ } else {
+ matcher = unorderedListPattern.matcher(paramType);
+ if (matcher.find()) {
+ String subType = paramType.substring(paramType.indexOf("{{") + 2, paramType.lastIndexOf("}}"));
+ return new AUnorderedListType(getTypeInfo(subType, txnCtx, function), "AUnorderedList");
+ }
+ }
+ return null;
+ }
+
+ private static IResultTypeComputer getResultTypeComputer(final MetadataTransactionContext txnCtx,
+ final Function function) throws MetadataException {
+
+ final IAType type = getTypeInfo(function.getReturnType(), txnCtx, function);
+ switch (type.getTypeTag()) {
+ case INT32:
+ return AInt32TypeComputer.INSTANCE;
+ case FLOAT:
+ return AFloatTypeComputer.INSTANCE;
+ case DOUBLE:
+ return ADoubleTypeComputer.INSTANCE;
+ case STRING:
+ return AStringTypeComputer.INSTANCE;
+ case ORDEREDLIST:
+ return new IResultTypeComputer() {
+ @Override
+ public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+ IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+
+ return new AOrderedListType(((AOrderedListType) type).getItemType(), ((AOrderedListType) type)
+ .getItemType().getTypeName());
+ }
+
+ };
+ case UNORDEREDLIST:
+ return new IResultTypeComputer() {
+ @Override
+ public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+ IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+
+ return new AUnorderedListType(type, type.getTypeName());
+ }
+
+ };
+ default:
+ IResultTypeComputer typeComputer = new IResultTypeComputer() {
+ @Override
+ public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+ IMetadataProvider<?, ?> mp) throws AlgebricksException {
+ return type;
+ }
+ };
+ return typeComputer;
+ }
+
+ }
+
+ private static IAType getType(Function function, MetadataTransactionContext txnCtx) throws AlgebricksException {
+ IAType collectionType = null;
+ try {
+ collectionType = getCollectionType(function.getReturnType(), txnCtx, function);
+ if (collectionType != null) {
+ return collectionType;
+ } else {
+
+ Datatype datatype;
+ datatype = MetadataManager.INSTANCE.getDatatype(txnCtx, function.getDataverseName(),
+ function.getReturnType());
+ return datatype.getDatatype();
+ }
+ } catch (MetadataException me) {
+ throw new AlgebricksException(me);
+ }
+ }
+
+ private static IFunctionInfo getUnnestFunctionInfo(MetadataTransactionContext txnCtx, Function function) {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ private static IFunctionInfo getStatefulFunctionInfo(MetadataTransactionContext txnCtx, Function function) {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ private static IFunctionInfo getAggregateFunctionInfo(MetadataTransactionContext txnCtx, Function function) {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ public static void main(String args[]) throws FileNotFoundException, IOException {
+ ExternalFunctionCompilerUtil obj = new ExternalFunctionCompilerUtil();
+ ObjectOutputStream oos = new ObjectOutputStream(new FileOutputStream("/tmp/ecu.obj"));
+ oos.writeObject(obj);
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java
index d5d4cc2..5ab32b5 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java
@@ -40,6 +40,8 @@
AsterixBuiltinFunctions.addDatasetFunction(AsterixBuiltinFunctions.DATASET);
AsterixBuiltinFunctions.addUnnestFun(AsterixBuiltinFunctions.FEED_INGEST, false);
AsterixBuiltinFunctions.addDatasetFunction(AsterixBuiltinFunctions.FEED_INGEST);
+ AsterixBuiltinFunctions.addUnnestFun(AsterixBuiltinFunctions.FEED_INTERCEPT, false);
+ AsterixBuiltinFunctions.addDatasetFunction(AsterixBuiltinFunctions.FEED_INTERCEPT);
}
public static void addMetadataBuiltinFunctions() {
@@ -93,6 +95,44 @@
public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
IMetadataProvider<?, ?> mp) throws AlgebricksException {
AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expression;
+ if (f.getArguments().size() != 3) {
+ throw new AlgebricksException("Incorrect number of arguments -> arity is 3, not "
+ + f.getArguments().size());
+ }
+ ILogicalExpression a1 = f.getArguments().get(1).getValue();
+ IAType t1 = (IAType) env.getType(a1);
+ if (t1.getTypeTag() == ATypeTag.ANY) {
+ return BuiltinType.ANY;
+ }
+ if (t1.getTypeTag() != ATypeTag.STRING) {
+ throw new AlgebricksException("Illegal type " + t1 + " for feed-ingest argument.");
+ }
+ if (a1.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+ return BuiltinType.ANY;
+ }
+ AsterixConstantValue acv = (AsterixConstantValue) ((ConstantExpression) a1).getValue();
+ String typeArg = ((AString) acv.getObject()).getStringValue();
+ AqlMetadataProvider metadata = ((AqlMetadataProvider) mp);
+ Pair<String, String> argInfo = getDatasetInfo(metadata, typeArg);
+ String dataverseName = argInfo.first;
+ String typeName = argInfo.second;
+ if (dataverseName == null) {
+ throw new AlgebricksException("Unspecified dataverse!");
+ }
+ IAType t2 = metadata.findType(dataverseName, typeName);
+ if (t2 == null) {
+ throw new AlgebricksException("Unknown type " + typeName);
+ }
+ return t2;
+ }
+ });
+
+ AsterixBuiltinFunctions.addFunction(AsterixBuiltinFunctions.FEED_INTERCEPT, new IResultTypeComputer() {
+
+ @Override
+ public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+ IMetadataProvider<?, ?> mp) throws AlgebricksException {
+ AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expression;
if (f.getArguments().size() != 1) {
throw new AlgebricksException("dataset arity is 1, not " + f.getArguments().size());
}
@@ -132,17 +172,16 @@
}
private static Pair<String, String> getDatasetInfo(AqlMetadataProvider metadata, String datasetArg) {
- String[] datasetNameComponents = datasetArg.split("\\.");
- String dataverseName;
- String datasetName;
- if (datasetNameComponents.length == 1) {
- dataverseName = metadata.getDefaultDataverse() == null ? null : metadata.getDefaultDataverse()
- .getDataverseName();
- datasetName = datasetNameComponents[0];
+ String[] nameComponents = datasetArg.split("\\.");
+ String first;
+ String second;
+ if (nameComponents.length == 1) {
+ first = metadata.getDefaultDataverse() == null ? null : metadata.getDefaultDataverse().getDataverseName();
+ second = nameComponents[0];
} else {
- dataverseName = datasetNameComponents[0];
- datasetName = datasetNameComponents[1];
+ first = nameComponents[0];
+ second = nameComponents[1];
}
- return new Pair(dataverseName, datasetName);
+ return new Pair(first, second);
}
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AqlExpressionTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AqlExpressionTypeComputer.java
index 7eb2f30..12b2753 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AqlExpressionTypeComputer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AqlExpressionTypeComputer.java
@@ -17,8 +17,10 @@
import java.util.ArrayList;
import java.util.List;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.AsterixExternalFunctionInfo;
import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
import edu.uci.ics.asterix.om.types.AUnionType;
import edu.uci.ics.asterix.om.types.BuiltinType;
@@ -72,8 +74,14 @@
unionList.add(BuiltinType.ABOOLEAN);
return new AUnionType(unionList, "OptionalBoolean");
}
- // Note: only builtin functions, for now.
- IResultTypeComputer rtc = AsterixBuiltinFunctions.getResultTypeComputer(fi);
+ // Note: built-in functions + udfs
+ IResultTypeComputer rtc = null;
+ FunctionSignature signature = new FunctionSignature(fi.getNamespace(), fi.getName(), fi.getArity());
+ if (AsterixBuiltinFunctions.isBuiltinCompilerFunction(signature, true)) {
+ rtc = AsterixBuiltinFunctions.getResultTypeComputer(fi);
+ } else {
+ rtc = ((AsterixExternalFunctionInfo) expr.getFunctionInfo()).getResultTypeComputer();
+ }
if (rtc == null) {
throw new AlgebricksException("Type computer missing for " + fi);
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AMutableUnorderedList.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AMutableUnorderedList.java
index fe0086b..7cb490c 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AMutableUnorderedList.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AMutableUnorderedList.java
@@ -20,15 +20,20 @@
public final class AMutableUnorderedList extends AUnorderedList {
- public AMutableUnorderedList(AUnorderedListType type) {
- super(type);
- }
+ public AMutableUnorderedList(AUnorderedListType type) {
+ super(type);
+ }
- public AMutableUnorderedList(AUnorderedListType type, ArrayList<IAObject> sequence) {
- super(type, sequence);
- }
+ public AMutableUnorderedList(AUnorderedListType type,
+ ArrayList<IAObject> sequence) {
+ super(type, sequence);
+ }
- public void add(IAObject obj) {
- values.add(obj);
- }
+ public void add(IAObject obj) {
+ values.add(obj);
+ }
+
+ public void clear() {
+ values.clear();
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
index 3c8a0c4..540530e 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
@@ -49,6 +49,7 @@
import edu.uci.ics.asterix.om.typecomputer.impl.CollectionToSequenceTypeComputer;
import edu.uci.ics.asterix.om.typecomputer.impl.ConcatNonNullTypeComputer;
import edu.uci.ics.asterix.om.typecomputer.impl.FieldAccessByIndexResultType;
+import edu.uci.ics.asterix.om.typecomputer.impl.FlowRecordResultTypeComputer;
import edu.uci.ics.asterix.om.typecomputer.impl.InjectFailureTypeComputer;
import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedCollectionMemberResultType;
import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedFieldAccessByNameResultType;
@@ -239,7 +240,9 @@
public final static FunctionIdentifier DATASET = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "dataset", 1);
public final static FunctionIdentifier FEED_INGEST = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
- "feed-ingest", 1);
+ "feed-ingest", 3);
+ public final static FunctionIdentifier FEED_INTERCEPT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "feed-intercept", 1);
public final static FunctionIdentifier INDEX_SEARCH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
"index-search", FunctionIdentifier.VARARGS);
@@ -494,6 +497,8 @@
"inject-failure", 2);
public final static FunctionIdentifier CAST_RECORD = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
"cast-record", 1);
+ public final static FunctionIdentifier FLOW_RECORD = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "flow-record", 1);
public final static FunctionIdentifier CAST_LIST = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
"cast-list", 1);
@@ -811,7 +816,6 @@
addPrivateFunction(TYPE_OF, null);
addPrivateFunction(UNORDERED_LIST_CONSTRUCTOR, UnorderedListConstructorResultType.INSTANCE);
addFunction(WORD_TOKENS, new IResultTypeComputer() {
-
@Override
public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
IMetadataProvider<?, ?> mp) throws AlgebricksException {
@@ -960,12 +964,14 @@
static {
datasetFunctions.add(getAsterixFunctionInfo(DATASET));
datasetFunctions.add(getAsterixFunctionInfo(FEED_INGEST));
+ datasetFunctions.add(getAsterixFunctionInfo(FEED_INTERCEPT));
datasetFunctions.add(getAsterixFunctionInfo(INDEX_SEARCH));
}
static {
addUnnestFun(DATASET, false);
addUnnestFun(FEED_INGEST, false);
+ addUnnestFun(FEED_INTERCEPT, false);
addUnnestFun(RANGE, true);
addUnnestFun(SCAN_COLLECTION, false);
addUnnestFun(SUBSET_COLLECTION, false);
@@ -979,9 +985,10 @@
return datasetFunctions.contains(getAsterixFunctionInfo(fi));
}
+ /*
public static boolean isBuiltinCompilerFunction(FunctionIdentifier fi, boolean includePrivateFunctions) {
return builtinPublicFunctionsSet.keySet().contains(getAsterixFunctionInfo(fi));
- }
+ }*/
public static boolean isBuiltinCompilerFunction(FunctionSignature signature, boolean includePrivateFunctions) {
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixFunctionInfo.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixFunctionInfo.java
index 186ca78..c536a88 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixFunctionInfo.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixFunctionInfo.java
@@ -14,11 +14,13 @@
*/
package edu.uci.ics.asterix.om.functions;
+import java.io.Serializable;
+
import edu.uci.ics.asterix.common.functions.FunctionSignature;
import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
-public class AsterixFunctionInfo implements IFunctionInfo {
+public class AsterixFunctionInfo implements IFunctionInfo, Serializable {
private final FunctionIdentifier functionIdentifier;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/base/IResultTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/base/IResultTypeComputer.java
index bd75f9c..3bc7792 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/base/IResultTypeComputer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/base/IResultTypeComputer.java
@@ -14,6 +14,7 @@
*/
package edu.uci.ics.asterix.om.typecomputer.base;
+import java.io.Serializable;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -21,7 +22,7 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
-public interface IResultTypeComputer {
+public interface IResultTypeComputer extends Serializable {
public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException;
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/FlowRecordResultTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/FlowRecordResultTypeComputer.java
new file mode 100644
index 0000000..a982a1f
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/FlowRecordResultTypeComputer.java
@@ -0,0 +1,26 @@
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.base.TypeComputerUtilities;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+public class FlowRecordResultTypeComputer implements IResultTypeComputer {
+
+ public static final FlowRecordResultTypeComputer INSTANCE = new FlowRecordResultTypeComputer();
+
+ @Override
+ public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+ IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+ ScalarFunctionCallExpression funcExpr = (ScalarFunctionCallExpression) expression;
+ IAType type = TypeComputerUtilities.getRequiredType(funcExpr);
+ if (type == null) {
+ type = (IAType) env.getType(funcExpr.getArguments().get(0).getValue());
+ }
+ return type;
+ }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java
index 0f836af..b8b3226 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java
@@ -27,6 +27,7 @@
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
@@ -46,8 +47,9 @@
private AsterixMetadataProperties metadataProperties;
private AsterixStorageProperties storageProperties;
private AsterixTransactionProperties txnProperties;
+ private IHyracksClientConnection hcc;
- public static void initialize(ICCApplicationContext ccAppCtx) throws AsterixException {
+ public static void initialize(ICCApplicationContext ccAppCtx, IHyracksClientConnection hcc) throws AsterixException {
if (INSTANCE == null) {
INSTANCE = new AsterixAppContextInfo(ccAppCtx);
}
@@ -57,6 +59,7 @@
INSTANCE.metadataProperties = new AsterixMetadataProperties(propertiesAccessor);
INSTANCE.storageProperties = new AsterixStorageProperties(propertiesAccessor);
INSTANCE.txnProperties = new AsterixTransactionProperties(propertiesAccessor);
+ INSTANCE.hcc = hcc;
Logger.getLogger("edu.uci.ics").setLevel(INSTANCE.externalProperties.getLogLevel());
}
@@ -98,6 +101,10 @@
return externalProperties;
}
+ public IHyracksClientConnection getHcc() {
+ return hcc;
+ }
+
@Override
public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider() {
return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
@@ -107,4 +114,5 @@
public IStorageManagerInterface getStorageManagerInterface() {
return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
}
+
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixClusterProperties.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixClusterProperties.java
index 6e6e09a..9af206bb 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixClusterProperties.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixClusterProperties.java
@@ -14,11 +14,25 @@
*/
package edu.uci.ics.asterix.om.util;
+import java.io.InputStream;
+import java.util.ArrayList;
import java.util.HashMap;
+import java.util.List;
import java.util.Map;
+import java.util.Set;
+import java.util.HashSet;
import java.util.logging.Level;
import java.util.logging.Logger;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Unmarshaller;
+
+import edu.uci.ics.asterix.event.schema.cluster.Cluster;
+import edu.uci.ics.asterix.event.schema.cluster.Node;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+
/**
* A holder class for properties related to the Asterix cluster.
*/
@@ -27,13 +41,31 @@
private static final Logger LOGGER = Logger.getLogger(AsterixClusterProperties.class.getName());
- private static final String IO_DEVICES = "iodevices";
-
public static final AsterixClusterProperties INSTANCE = new AsterixClusterProperties();
+ public static final String CLUSTER_CONFIGURATION_FILE = "cluster.xml";
+
+ private static final String IO_DEVICES = "iodevices";
private Map<String, Map<String, String>> ncConfiguration = new HashMap<String, Map<String, String>>();
+ private final Cluster cluster;
+
+ private AlgebricksAbsolutePartitionConstraint clusterPartitionConstraint;
+
private AsterixClusterProperties() {
+ InputStream is = this.getClass().getClassLoader().getResourceAsStream(CLUSTER_CONFIGURATION_FILE);
+ if (is != null) {
+ try {
+ JAXBContext ctx = JAXBContext.newInstance(Cluster.class);
+ Unmarshaller unmarshaller = ctx.createUnmarshaller();
+ cluster = (Cluster) unmarshaller.unmarshal(is);
+
+ } catch (JAXBException e) {
+ throw new IllegalStateException("Failed to read configuration file " + CLUSTER_CONFIGURATION_FILE);
+ }
+ } else {
+ cluster = null;
+ }
}
public enum State {
@@ -43,12 +75,13 @@
private State state = State.UNUSABLE;
- public void removeNCConfiguration(String nodeId) {
- state = State.UNUSABLE;
+ public synchronized void removeNCConfiguration(String nodeId) {
+ // state = State.UNUSABLE;
ncConfiguration.remove(nodeId);
+ resetClusterPartitionConstraint();
}
- public void addNCConfiguration(String nodeId, Map<String, String> configuration) {
+ public synchronized void addNCConfiguration(String nodeId, Map<String, String> configuration) {
ncConfiguration.put(nodeId, configuration);
if (ncConfiguration.keySet().size() == AsterixAppContextInfo.getInstance().getMetadataProperties()
.getNodeNames().size()) {
@@ -57,6 +90,7 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(" Registering configuration parameters for node id" + nodeId);
}
+ resetClusterPartitionConstraint();
}
/**
@@ -64,10 +98,11 @@
*
* @param nodeId
* unique identifier of the Node Controller
- * @return number of IO devices. -1 if the node id is not valid. A node id is not valid
- * if it does not correspond to the set of registered Node Controllers.
+ * @return number of IO devices. -1 if the node id is not valid. A node id
+ * is not valid if it does not correspond to the set of registered
+ * Node Controllers.
*/
- public int getNumberOfIODevices(String nodeId) {
+ public synchronized int getNumberOfIODevices(String nodeId) {
Map<String, String> ncConfig = ncConfiguration.get(nodeId);
if (ncConfig == null) {
if (LOGGER.isLoggable(Level.WARNING)) {
@@ -78,7 +113,7 @@
}
return ncConfig.get(IO_DEVICES).split(",").length;
}
-
+
/**
* Returns the IO devices configured for a Node Controller
*
@@ -103,4 +138,105 @@
return state;
}
+ public Cluster getCluster() {
+ return cluster;
+ }
+
+ public synchronized Node getAvailableSubstitutionNode() {
+ List<Node> subNodes = cluster.getSubstituteNodes() == null ? null : cluster.getSubstituteNodes().getNode();
+ return subNodes == null || subNodes.isEmpty() ? null : subNodes.get(0);
+ }
+
+ public synchronized Set<String> getParticipantNodes() {
+ Set<String> participantNodes = new HashSet<String>();
+ for (String pNode : ncConfiguration.keySet()) {
+ participantNodes.add(pNode);
+ }
+ return participantNodes;
+ }
+
+ public synchronized AlgebricksPartitionConstraint getClusterLocations() {
+ if (clusterPartitionConstraint == null) {
+ resetClusterPartitionConstraint();
+ }
+ return clusterPartitionConstraint;
+ }
+
+ private synchronized void resetClusterPartitionConstraint() {
+ Map<String, String[]> stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
+ ArrayList<String> locs = new ArrayList<String>();
+ for (String i : stores.keySet()) {
+ String[] nodeStores = stores.get(i);
+ int numIODevices = AsterixClusterProperties.INSTANCE.getNumberOfIODevices(i);
+ for (int j = 0; j < nodeStores.length; j++) {
+ for (int k = 0; k < numIODevices; k++) {
+ locs.add(i);
+ }
+ }
+ }
+ String[] cluster = new String[locs.size()];
+ cluster = locs.toArray(cluster);
+ clusterPartitionConstraint = new AlgebricksAbsolutePartitionConstraint(cluster);
+ }
+
+ private static class AsterixCluster {
+
+ private final String asterixInstance;
+ private Map<String, AsterixNode> asterixNodes;
+
+ public AsterixCluster(Cluster cluster) {
+ asterixInstance = cluster.getInstanceName();
+ asterixNodes = new HashMap<String, AsterixNode>();
+ for (Node node : cluster.getNode()) {
+ AsterixNode aNode = new AsterixNode(node, AsterixNode.NodeRole.PARTICIPANT,
+ AsterixNode.NodeState.INACTIVE);
+ asterixNodes.put(asterixInstance + "_" + node.getId(), aNode);
+ }
+
+ for (Node node : cluster.getSubstituteNodes().getNode()) {
+ AsterixNode aNode = new AsterixNode(node, AsterixNode.NodeRole.SUBSTITUTE,
+ AsterixNode.NodeState.INACTIVE);
+ asterixNodes.put(asterixInstance + "_" + node.getId(), aNode);
+ }
+ }
+
+ private static class AsterixNode {
+
+ private final Node node;
+ private NodeRole role;
+ private NodeState state;
+
+ public enum NodeRole {
+ PARTICIPANT,
+ SUBSTITUTE
+ }
+
+ public enum NodeState {
+ ACTIVE,
+ INACTIVE
+ }
+
+ public AsterixNode(Node node, NodeRole role, NodeState state) {
+ this.node = node;
+ this.role = role;
+ this.state = state;
+ }
+
+ @Override
+ public String toString() {
+ return node.getId() + "_" + role + "_" + state;
+ }
+ }
+
+ public void notifyChangeState(String nodeId, AsterixNode.NodeRole newRole, AsterixNode.NodeState newState) {
+ AsterixNode node = asterixNodes.get(nodeId);
+ if (node != null) {
+ node.role = newRole;
+ node.state = newState;
+ } else {
+ throw new IllegalStateException("Unknown nodeId" + nodeId);
+ }
+
+ }
+ }
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/FlowRecordDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/FlowRecordDescriptor.java
new file mode 100644
index 0000000..81c23cd
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/FlowRecordDescriptor.java
@@ -0,0 +1,74 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.pointables.PointableAllocator;
+import edu.uci.ics.asterix.om.pointables.base.IVisitablePointable;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class FlowRecordDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new FlowRecordDescriptor();
+ }
+ };
+
+ private static final long serialVersionUID = 1L;
+ private ARecordType inputType;
+
+ public void reset(ARecordType inputType) {
+ this.inputType = inputType;
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return AsterixBuiltinFunctions.FLOW_RECORD;
+ }
+
+ @Override
+ public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+ final ICopyEvaluatorFactory recordEvalFactory = args[0];
+
+ return new ICopyEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ final DataOutput out = output.getDataOutput();
+ final ArrayBackedValueStorage recordBuffer = new ArrayBackedValueStorage();
+ final ICopyEvaluator recEvaluator = recordEvalFactory.createEvaluator(recordBuffer);
+
+ return new ICopyEvaluator() {
+ // pointable allocator
+ private PointableAllocator allocator = new PointableAllocator();
+ final IVisitablePointable recAccessor = allocator.allocateRecordValue(inputType);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ try {
+ recordBuffer.reset();
+ recEvaluator.evaluate(tuple);
+ recAccessor.set(recordBuffer);
+ out.write(recAccessor.getByteArray(), recAccessor.getStartOffset(), recAccessor.getLength());
+ } catch (Exception ioe) {
+ throw new AlgebricksException(ioe);
+ }
+ }
+ };
+ }
+ };
+ }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
index 566aa98..011836a 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -151,6 +151,7 @@
import edu.uci.ics.asterix.runtime.evaluators.functions.EndsWithDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.FieldAccessByIndexDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.FieldAccessByNameDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.FlowRecordDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.FuzzyEqDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.GetItemDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.GramTokensDescriptor;
@@ -494,6 +495,7 @@
temp.add(InjectFailureDescriptor.FACTORY);
temp.add(CastListDescriptor.FACTORY);
temp.add(CastRecordDescriptor.FACTORY);
+ temp.add(FlowRecordDescriptor.FACTORY);
temp.add(NotNullDescriptor.FACTORY);
// Spatial and temporal type accessors
@@ -741,6 +743,10 @@
}
((CastListDescriptor) fd).reset(rt, (AbstractCollectionType) it);
}
+ if (fd.getIdentifier().equals(AsterixBuiltinFunctions.FLOW_RECORD)) {
+ ARecordType it = (ARecordType) TypeComputerUtilities.getInputType((AbstractFunctionCallExpression) expr);
+ ((FlowRecordDescriptor) fd).reset(it);
+ }
if (fd.getIdentifier().equals(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR)) {
ARecordType rt = (ARecordType) context.getType(expr);
((OpenRecordConstructorDescriptor) fd).reset(rt,
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
index 2322338..ab52939 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
@@ -18,6 +18,8 @@
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.types.ARecordType;
@@ -36,6 +38,8 @@
*/
public abstract class AbstractTupleParser implements ITupleParser {
+ protected static Logger LOGGER = Logger.getLogger(AbstractTupleParser.class.getName());
+
protected ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
protected DataOutput dos = tb.getDataOutput();
protected final FrameTupleAppender appender;
@@ -68,6 +72,7 @@
addTupleToFrame(writer);
}
if (appender.getTupleCount() > 0) {
+
FrameUtils.flushFrame(frame, writer);
}
} catch (AsterixException ae) {
diff --git a/asterix-test-framework/pom.xml b/asterix-test-framework/pom.xml
old mode 100755
new mode 100644
diff --git a/asterix-test-framework/src/main/resources/Catalog.xsd b/asterix-test-framework/src/main/resources/Catalog.xsd
old mode 100755
new mode 100644
diff --git a/asterix-tools/data/uscensus/dist.all.first.cleaned b/asterix-tools/data/uscensus/dist.all.first.cleaned
old mode 100755
new mode 100644
diff --git a/asterix-tools/data/uscensus/dist.all.last.cleaned b/asterix-tools/data/uscensus/dist.all.last.cleaned
old mode 100755
new mode 100644
diff --git a/asterix-tools/data/uscensus/dist.female.first.cleaned b/asterix-tools/data/uscensus/dist.female.first.cleaned
old mode 100755
new mode 100644
diff --git a/asterix-tools/data/uscensus/dist.male.first.cleaned b/asterix-tools/data/uscensus/dist.male.first.cleaned
old mode 100755
new mode 100644
diff --git a/asterix-tools/pom.xml b/asterix-tools/pom.xml
index 6c7db41..861dd79 100644
--- a/asterix-tools/pom.xml
+++ b/asterix-tools/pom.xml
@@ -148,6 +148,12 @@
<scope>compile</scope>
</dependency>
<dependency>
+ <groupId>edu.uci.ics.asterix</groupId>
+ <artifactId>asterix-external-data</artifactId>
+ <version>0.8.1-SNAPSHOT</version>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>
<version>4.2.2</version>
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGenerator.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGenerator.java
new file mode 100644
index 0000000..0c93564
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGenerator.java
@@ -0,0 +1,2476 @@
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+import java.util.UUID;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.OutputKeys;
+import javax.xml.transform.Transformer;
+import javax.xml.transform.TransformerException;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.transform.stream.StreamResult;
+
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+public class DataGenerator {
+
+ private static RandomDateGenerator randDateGen;
+ private static RandomNameGenerator randNameGen;
+ private static RandomEmploymentGenerator randEmpGen;
+ private static RandomMessageGenerator randMessageGen;
+ private static RandomLocationGenerator randLocationGen;
+
+ private static DistributionHandler fbDistHandler;
+ private static DistributionHandler twDistHandler;
+
+ private static int totalFbMessages;
+ private static int numFbOnlyUsers;
+ private static int totalTwMessages;
+ private static int numTwOnlyUsers;
+
+ private static int numCommonUsers;
+
+ private static int fbUserId;
+ private static int twUserId;
+
+ private static int fbMessageId;
+ private static int twMessageId;
+
+ private static Random random = new Random();
+
+ private static String commonUserFbSuffix = "_fb";
+ private static String commonUserTwSuffix = "_tw";
+
+ private static String outputDir;
+
+ private static PartitionConfiguration partition;
+
+ private static FacebookUser fbUser = new FacebookUser();
+ private static TwitterUser twUser = new TwitterUser();
+
+ private static FacebookMessage fbMessage = new FacebookMessage();
+ private static TweetMessage twMessage = new TweetMessage();
+
+ private static int duration;
+
+ private static DateFormat dateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss");
+
+ private static void generateFacebookOnlyUsers(int numFacebookUsers) throws IOException {
+ FileAppender appender = FileUtil.getFileAppender(outputDir + "/" + "fb_users.adm", true, true);
+ FileAppender messageAppender = FileUtil.getFileAppender(outputDir + "/" + "fb_message.adm", true, true);
+
+ for (int i = 0; i < numFacebookUsers; i++) {
+ getFacebookUser(null);
+ appender.appendToFile(fbUser.toString());
+ generateFacebookMessages(fbUser, messageAppender, -1);
+ }
+ appender.close();
+ messageAppender.close();
+ }
+
+ private static void generateTwitterOnlyUsers(int numTwitterUsers) throws IOException {
+ FileAppender appender = FileUtil.getFileAppender(outputDir + "/" + "tw_users.adm", true, true);
+ FileAppender messageAppender = FileUtil.getFileAppender(outputDir + "/" + "tw_message.adm", true, true);
+
+ for (int i = 0; i < numTwitterUsers; i++) {
+ getTwitterUser(null);
+ appender.appendToFile(twUser.toString());
+ generateTwitterMessages(twUser, messageAppender, -1);
+ }
+ appender.close();
+ messageAppender.close();
+ }
+
+ private static void generateCommonUsers() throws IOException {
+ FileAppender fbAppender = FileUtil.getFileAppender(outputDir + "/" + "fb_users.adm", true, false);
+ FileAppender twAppender = FileUtil.getFileAppender(outputDir + "/" + "tw_users.adm", true, false);
+ FileAppender fbMessageAppender = FileUtil.getFileAppender(outputDir + "/" + "fb_message.adm", true, false);
+ FileAppender twMessageAppender = FileUtil.getFileAppender(outputDir + "/" + "tw_message.adm", true, false);
+
+ for (int i = 0; i < numCommonUsers; i++) {
+ getFacebookUser(commonUserFbSuffix);
+ fbAppender.appendToFile(fbUser.toString());
+ generateFacebookMessages(fbUser, fbMessageAppender, -1);
+
+ getCorrespondingTwitterUser(fbUser);
+ twAppender.appendToFile(twUser.toString());
+ generateTwitterMessages(twUser, twMessageAppender, -1);
+ }
+
+ fbAppender.close();
+ twAppender.close();
+ fbMessageAppender.close();
+ twMessageAppender.close();
+ }
+
+ private static void generateFacebookMessages(FacebookUser user, FileAppender appender, int numMsg)
+ throws IOException {
+ Message message;
+ int numMessages = 0;
+ if (numMsg == -1) {
+ numMessages = fbDistHandler
+ .getFromDistribution(fbUserId - partition.getTargetPartition().getFbUserKeyMin());
+ }
+ for (int i = 0; i < numMessages; i++) {
+ message = randMessageGen.getNextRandomMessage();
+ Point location = randLocationGen.getRandomPoint();
+ fbMessage.reset(fbMessageId++, user.getId(), random.nextInt(totalFbMessages + 1), location, message);
+ appender.appendToFile(fbMessage.toString());
+ }
+ }
+
+ private static void generateTwitterMessages(TwitterUser user, FileAppender appender, int numMsg) throws IOException {
+ Message message;
+ int numMessages = 0;
+ if (numMsg == -1) {
+ numMessages = twDistHandler
+ .getFromDistribution(twUserId - partition.getTargetPartition().getTwUserKeyMin());
+ }
+
+ for (int i = 0; i < numMessages; i++) {
+ message = randMessageGen.getNextRandomMessage();
+ Point location = randLocationGen.getRandomPoint();
+ DateTime sendTime = randDateGen.getNextRandomDatetime();
+ twMessage.reset(twMessageId + "", user, location, sendTime, message.getReferredTopics(), message);
+ twMessageId++;
+ appender.appendToFile(twMessage.toString());
+ }
+ }
+
+ public static Iterator<TweetMessage> getTwitterMessageIterator() {
+ return new TweetMessageIterator(duration);
+ }
+
+ public static class TweetMessageIterator implements Iterator<TweetMessage> {
+
+ private final int duration;
+ private long startTime = 0;
+
+ public TweetMessageIterator(int duration) {
+ this.duration = duration;
+ }
+
+ @Override
+ public boolean hasNext() {
+ if (startTime == 0) {
+ startTime = System.currentTimeMillis();
+ }
+ return System.currentTimeMillis() - startTime < duration * 1000;
+ }
+
+ @Override
+ public TweetMessage next() {
+ getTwitterUser(null);
+ Message message = randMessageGen.getNextRandomMessage();
+ Point location = randLocationGen.getRandomPoint();
+ DateTime sendTime = randDateGen.getNextRandomDatetime();
+ twMessage.reset(UUID.randomUUID().toString(), twUser, location, sendTime, message.getReferredTopics(),
+ message);
+ twMessageId++;
+ if (twUserId > numTwOnlyUsers) {
+ twUserId = 1;
+ }
+ return twMessage;
+
+ }
+
+ @Override
+ public void remove() {
+ // TODO Auto-generated method stub
+
+ }
+
+ }
+
+ public static class InitializationInfo {
+ public Date startDate = new Date(1, 1, 2005);
+ public Date endDate = new Date(8, 20, 2012);
+ public String[] lastNames = DataGenerator.lastNames;
+ public String[] firstNames = DataGenerator.firstNames;
+ public String[] vendors = DataGenerator.vendors;
+ public String[] jargon = DataGenerator.jargon;
+ public String[] org_list = DataGenerator.org_list;
+ public int percentEmployed = 90;
+ public Date employmentStartDate = new Date(1, 1, 2000);
+ public Date employmentEndDate = new Date(31, 12, 2012);
+ public int totalFbMessages;
+ public int numFbOnlyUsers;
+ public int totalTwMessages;
+ public int numTwOnlyUsers = 5000;
+ public int numCommonUsers;
+ public int fbUserIdMin;
+ public int fbMessageIdMin;
+ public int twUserIdMin;
+ public int twMessageIdMin;
+ public int timeDurationInSecs = 60;
+
+ }
+
+ public static void initialize(InitializationInfo info) {
+ randDateGen = new RandomDateGenerator(info.startDate, info.endDate);
+ randNameGen = new RandomNameGenerator(info.firstNames, info.lastNames);
+ randEmpGen = new RandomEmploymentGenerator(info.percentEmployed, info.employmentStartDate,
+ info.employmentEndDate, info.org_list);
+ randLocationGen = new RandomLocationGenerator(24, 49, 66, 98);
+ randMessageGen = new RandomMessageGenerator(info.vendors, info.jargon);
+ fbDistHandler = new DistributionHandler(info.totalFbMessages, 0.5, info.numFbOnlyUsers + info.numCommonUsers);
+ twDistHandler = new DistributionHandler(info.totalTwMessages, 0.5, info.numTwOnlyUsers + info.numCommonUsers);
+ fbUserId = info.fbUserIdMin;
+ twUserId = info.twUserIdMin;
+
+ fbMessageId = info.fbMessageIdMin;
+ twMessageId = info.fbMessageIdMin;
+ duration = info.timeDurationInSecs;
+ }
+
+ public static void main(String args[]) throws Exception {
+
+ String controllerInstallDir = null;
+ if (args.length < 2) {
+ printUsage();
+ System.exit(1);
+ } else {
+ controllerInstallDir = args[0];
+ String partitionConfXML = controllerInstallDir + "/output/partition-conf.xml";
+ String partitionName = args[1];
+ partition = XMLUtil.getPartitionConfiguration(partitionConfXML, partitionName);
+ }
+
+ // 1
+ randDateGen = new RandomDateGenerator(new Date(1, 1, 2005), new Date(8, 20, 2012));
+
+ String firstNameFile = controllerInstallDir + "/metadata/firstNames.txt";
+ String lastNameFile = controllerInstallDir + "/metadata/lastNames.txt";
+ String vendorFile = controllerInstallDir + "/metadata/vendors.txt";
+ String jargonFile = controllerInstallDir + "/metadata/jargon.txt";
+ String orgList = controllerInstallDir + "/metadata/org_list.txt";
+
+ randNameGen = new RandomNameGenerator(firstNameFile, lastNameFile);
+ randEmpGen = new RandomEmploymentGenerator(90, new Date(1, 1, 2000), new Date(8, 20, 2012), orgList);
+ randLocationGen = new RandomLocationGenerator(24, 49, 66, 98);
+ randMessageGen = new RandomMessageGenerator(vendorFile, jargonFile);
+
+ totalFbMessages = partition.getTargetPartition().getFbMessageIdMax()
+ - partition.getTargetPartition().getFbMessageIdMin() + 1;
+ numFbOnlyUsers = (partition.getTargetPartition().getFbUserKeyMax()
+ - partition.getTargetPartition().getFbUserKeyMin() + 1)
+ - partition.getTargetPartition().getCommonUsers();
+
+ totalTwMessages = partition.getTargetPartition().getTwMessageIdMax()
+ - partition.getTargetPartition().getTwMessageIdMin() + 1;
+ numTwOnlyUsers = (partition.getTargetPartition().getTwUserKeyMax()
+ - partition.getTargetPartition().getTwUserKeyMin() + 1)
+ - partition.getTargetPartition().getCommonUsers();
+
+ numCommonUsers = partition.getTargetPartition().getCommonUsers();
+ fbDistHandler = new DistributionHandler(totalFbMessages, 0.5, numFbOnlyUsers + numCommonUsers);
+ twDistHandler = new DistributionHandler(totalTwMessages, 0.5, numTwOnlyUsers + numCommonUsers);
+
+ fbUserId = partition.getTargetPartition().getFbUserKeyMin();
+ twUserId = partition.getTargetPartition().getTwUserKeyMin();
+
+ fbMessageId = partition.getTargetPartition().getFbMessageIdMin();
+ twMessageId = partition.getTargetPartition().getTwMessageIdMin();
+
+ outputDir = partition.getSourcePartition().getPath();
+ generateData();
+ }
+
+ public static void printUsage() {
+ System.out.println(" Error: Invalid number of arguments ");
+ System.out.println(" Usage :" + " DataGenerator <path to configuration file> <partition name> ");
+ }
+
+ public static void generateData() throws IOException {
+ generateFacebookOnlyUsers(numFbOnlyUsers);
+ generateTwitterOnlyUsers(numTwOnlyUsers);
+ generateCommonUsers();
+ System.out.println("Partition :" + partition.getTargetPartition().getName() + " finished");
+ }
+
+ public static void getFacebookUser(String usernameSuffix) {
+ String suggestedName = randNameGen.getRandomName();
+ String[] nameComponents = suggestedName.split(" ");
+ String name = nameComponents[0] + nameComponents[1];
+ if (usernameSuffix != null) {
+ name = name + usernameSuffix;
+ }
+ String alias = nameComponents[0];
+ String userSince = randDateGen.getNextRandomDatetime().toString();
+ int numFriends = random.nextInt(25);
+ int[] friendIds = RandomUtil.getKFromN(numFriends, (numFbOnlyUsers + numCommonUsers));
+ Employment emp = randEmpGen.getRandomEmployment();
+ fbUser.reset(fbUserId++, alias, name, userSince, friendIds, emp);
+ }
+
+ public static void getTwitterUser(String usernameSuffix) {
+ String suggestedName = randNameGen.getRandomName();
+ String[] nameComponents = suggestedName.split(" ");
+ String screenName = nameComponents[0] + nameComponents[1] + randNameGen.getRandomNameSuffix();
+ String name = suggestedName;
+ if (usernameSuffix != null) {
+ name = name + usernameSuffix;
+ }
+ int numFriends = random.nextInt((int) (100)); // draw from Zipfian
+ int statusesCount = random.nextInt(500); // draw from Zipfian
+ int followersCount = random.nextInt((int) (200));
+ twUser.reset(screenName, numFriends, statusesCount, name, followersCount);
+ twUserId++;
+ }
+
+ public static void getCorrespondingTwitterUser(FacebookUser fbUser) {
+ String screenName = fbUser.getName().substring(0, fbUser.getName().lastIndexOf(commonUserFbSuffix))
+ + commonUserTwSuffix;
+ String name = screenName.split(" ")[0];
+ int numFriends = random.nextInt((int) ((numTwOnlyUsers + numCommonUsers)));
+ int statusesCount = random.nextInt(500); // draw from Zipfian
+ int followersCount = random.nextInt((int) (numTwOnlyUsers + numCommonUsers));
+ twUser.reset(screenName, numFriends, statusesCount, name, followersCount);
+ }
+
+ public static class RandomDateGenerator {
+
+ private final Date startDate;
+ private final Date endDate;
+ private final Random random = new Random();
+ private final int yearDifference;
+ private Date workingDate;
+ private Date recentDate;
+ private DateTime dateTime;
+
+ public RandomDateGenerator(Date startDate, Date endDate) {
+ this.startDate = startDate;
+ this.endDate = endDate;
+ yearDifference = endDate.getYear() - startDate.getYear() + 1;
+ workingDate = new Date();
+ recentDate = new Date();
+ dateTime = new DateTime();
+ }
+
+ public Date getStartDate() {
+ return startDate;
+ }
+
+ public Date getEndDate() {
+ return endDate;
+ }
+
+ public Date getNextRandomDate() {
+ int year = random.nextInt(yearDifference) + startDate.getYear();
+ int month;
+ int day;
+ if (year == endDate.getYear()) {
+ month = random.nextInt(endDate.getMonth()) + 1;
+ if (month == endDate.getMonth()) {
+ day = random.nextInt(endDate.getDay()) + 1;
+ } else {
+ day = random.nextInt(28) + 1;
+ }
+ } else {
+ month = random.nextInt(12) + 1;
+ day = random.nextInt(28) + 1;
+ }
+ workingDate.reset(month, day, year);
+ return workingDate;
+ }
+
+ public DateTime getNextRandomDatetime() {
+ Date randomDate = getNextRandomDate();
+ dateTime.reset(randomDate);
+ return dateTime;
+ }
+
+ public Date getNextRecentDate(Date date) {
+ int year = date.getYear()
+ + (date.getYear() == endDate.getYear() ? 0 : random.nextInt(endDate.getYear() - date.getYear()));
+ int month = (year == endDate.getYear()) ? date.getMonth() == endDate.getMonth() ? (endDate.getMonth())
+ : (date.getMonth() + random.nextInt(endDate.getMonth() - date.getMonth())) : random.nextInt(12) + 1;
+
+ int day = (year == endDate.getYear()) ? month == endDate.getMonth() ? date.getDay() == endDate.getDay() ? endDate
+ .getDay() : date.getDay() + random.nextInt(endDate.getDay() - date.getDay())
+ : random.nextInt(28) + 1
+ : random.nextInt(28) + 1;
+ recentDate.reset(month, day, year);
+ return recentDate;
+ }
+
+ public static void main(String args[]) throws Exception {
+ Date date = new Date(2, 20, 2012);
+ RandomDateGenerator dgen = new RandomDateGenerator(new Date(1, 1, 2005), new Date(8, 20, 2012));
+ while (true) {
+ Date nextDate = dgen.getNextRandomDate();
+ if (nextDate.getDay() == 0) {
+ throw new Exception("invalid date " + nextDate);
+ }
+
+ // System.out.println(" original date: " + date);
+ System.out.println(nextDate);
+ }
+ }
+ }
+
+ public static class DateTime extends Date {
+
+ private String hour = "10";
+ private String min = "10";
+ private String sec = "00";
+ private long chrononTime;
+
+ public DateTime(int month, int day, int year, String hour, String min, String sec) {
+ super(month, day, year);
+ this.hour = hour;
+ this.min = min;
+ this.sec = sec;
+ chrononTime = new java.util.Date(year, month, day, Integer.parseInt(hour), Integer.parseInt(min),
+ Integer.parseInt(sec)).getTime();
+ }
+
+ public void reset(int month, int day, int year, String hour, String min, String sec) {
+ super.setDay(month);
+ super.setDay(day);
+ super.setYear(year);
+ this.hour = hour;
+ this.min = min;
+ this.sec = sec;
+ chrononTime = new java.util.Date(year, month, day, Integer.parseInt(hour), Integer.parseInt(min),
+ Integer.parseInt(sec)).getTime();
+ }
+
+ public DateTime() {
+ }
+
+ public DateTime(Date date) {
+ super(date.getMonth(), date.getDay(), date.getYear());
+ }
+
+ public void reset(Date date) {
+ reset(date.getMonth(), date.getDay(), date.getYear());
+ }
+
+ public DateTime(Date date, int hour, int min, int sec) {
+ super(date.getMonth(), date.getDay(), date.getYear());
+ this.hour = (hour < 10) ? "0" : "" + hour;
+ this.min = (min < 10) ? "0" : "" + min;
+ this.sec = (sec < 10) ? "0" : "" + sec;
+ }
+
+ public long getChrononTime() {
+ return chrononTime;
+ }
+
+ public String getHour() {
+ return hour;
+ }
+
+ public String getMin() {
+ return min;
+ }
+
+ public String getSec() {
+ return sec;
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("datetime");
+ builder.append("(\"");
+ builder.append(super.getYear());
+ builder.append("-");
+ builder.append(super.getMonth() < 10 ? "0" + super.getMonth() : super.getMonth());
+ builder.append("-");
+ builder.append(super.getDay() < 10 ? "0" + super.getDay() : super.getDay());
+ builder.append("T");
+ builder.append(hour + ":" + min + ":" + sec);
+ builder.append("\")");
+ return builder.toString();
+ }
+ }
+
+ public static class Message {
+
+ private char[] message = new char[500];
+ private List<String> referredTopics;
+ private int length;
+
+ public Message(char[] m, List<String> referredTopics) {
+ System.arraycopy(m, 0, message, 0, m.length);
+ length = m.length;
+ this.referredTopics = referredTopics;
+ }
+
+ public Message() {
+ referredTopics = new ArrayList<String>();
+ length = 0;
+ }
+
+ public char[] getMessage() {
+ return message;
+ }
+
+ public List<String> getReferredTopics() {
+ return referredTopics;
+ }
+
+ public void reset(char[] m, int offset, int length, List<String> referredTopics) {
+ System.arraycopy(m, offset, message, 0, length);
+ this.length = length;
+ this.referredTopics = referredTopics;
+ }
+
+ public int getLength() {
+ return length;
+ }
+
+ public char charAt(int index) {
+ return message[index];
+ }
+
+ }
+
+ public static class Point {
+
+ private float latitude;
+ private float longitude;
+
+ public float getLatitude() {
+ return latitude;
+ }
+
+ public float getLongitude() {
+ return longitude;
+ }
+
+ public Point(float latitude, float longitude) {
+ this.latitude = latitude;
+ this.longitude = longitude;
+ }
+
+ public void reset(float latitude, float longitude) {
+ this.latitude = latitude;
+ this.longitude = longitude;
+ }
+
+ public Point() {
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("point(\"" + latitude + "," + longitude + "\")");
+ return builder.toString();
+ }
+ }
+
+ public static class RandomNameGenerator {
+
+ private String[] firstNames;
+ private String[] lastNames;
+
+ private final Random random = new Random();
+
+ private final String[] connectors = new String[] { "_", "#", "$", "@" };
+
+ public RandomNameGenerator(String firstNameFilePath, String lastNameFilePath) throws IOException {
+ firstNames = FileUtil.listyFile(new File(firstNameFilePath)).toArray(new String[] {});
+ lastNames = FileUtil.listyFile(new File(lastNameFilePath)).toArray(new String[] {});
+ }
+
+ public RandomNameGenerator(String[] firstNames, String[] lastNames) {
+ this.firstNames = firstNames;
+ this.lastNames = lastNames;
+ }
+
+ public String getRandomName() {
+ String name;
+ name = getSuggestedName();
+ return name;
+
+ }
+
+ private String getSuggestedName() {
+ int firstNameIndex = random.nextInt(firstNames.length);
+ int lastNameIndex = random.nextInt(lastNames.length);
+ String suggestedName = firstNames[firstNameIndex] + " " + lastNames[lastNameIndex];
+ return suggestedName;
+ }
+
+ public String getRandomNameSuffix() {
+ return connectors[random.nextInt(connectors.length)] + random.nextInt(1000);
+ }
+ }
+
+ public static class RandomMessageGenerator {
+
+ private final MessageTemplate messageTemplate;
+
+ public RandomMessageGenerator(String vendorFilePath, String jargonFilePath) throws IOException {
+ List<String> vendors = FileUtil.listyFile(new File(vendorFilePath));
+ List<String> jargon = FileUtil.listyFile(new File(jargonFilePath));
+ this.messageTemplate = new MessageTemplate(vendors, jargon);
+ }
+
+ public RandomMessageGenerator(String[] vendors, String[] jargon) {
+ List<String> vendorList = new ArrayList<String>();
+ for (String v : vendors) {
+ vendorList.add(v);
+ }
+ List<String> jargonList = new ArrayList<String>();
+ for (String j : jargon) {
+ jargonList.add(j);
+ }
+ this.messageTemplate = new MessageTemplate(vendorList, jargonList);
+ }
+
+ public Message getNextRandomMessage() {
+ return messageTemplate.getNextMessage();
+ }
+ }
+
+ public static class AbstractMessageTemplate {
+
+ protected final Random random = new Random();
+
+ protected String[] positiveVerbs = new String[] { "like", "love" };
+ protected String[] negativeVerbs = new String[] { "dislike", "hate", "can't stand" };
+
+ protected String[] negativeAdjectives = new String[] { "horrible", "bad", "terrible", "OMG" };
+ protected String[] postiveAdjectives = new String[] { "good", "awesome", "amazing", "mind-blowing" };
+
+ protected String[] otherWords = new String[] { "the", "its" };
+ }
+
+ public static class MessageTemplate extends AbstractMessageTemplate {
+
+ private List<String> vendors;
+ private List<String> jargon;
+ private CharBuffer buffer;
+ private List<String> referredTopics;
+ private Message message = new Message();
+
+ public MessageTemplate(List<String> vendors, List<String> jargon) {
+ this.vendors = vendors;
+ this.jargon = jargon;
+ buffer = CharBuffer.allocate(2500);
+ referredTopics = new ArrayList<String>();
+ }
+
+ public Message getNextMessage() {
+ buffer.position(0);
+ buffer.limit(2500);
+ referredTopics.clear();
+ boolean isPositive = random.nextBoolean();
+ String[] verbArray = isPositive ? positiveVerbs : negativeVerbs;
+ String[] adjectiveArray = isPositive ? postiveAdjectives : negativeAdjectives;
+ String verb = verbArray[random.nextInt(verbArray.length)];
+ String adjective = adjectiveArray[random.nextInt(adjectiveArray.length)];
+
+ buffer.put(" ");
+ buffer.put(verb);
+ buffer.put(" ");
+ String vendor = vendors.get(random.nextInt(vendors.size()));
+ referredTopics.add(vendor);
+ buffer.append(vendor);
+ buffer.append(" ");
+ buffer.append(otherWords[random.nextInt(otherWords.length)]);
+ buffer.append(" ");
+ String jargonTerm = jargon.get(random.nextInt(jargon.size()));
+ referredTopics.add(jargonTerm);
+ buffer.append(jargonTerm);
+ buffer.append(" is ");
+ buffer.append(adjective);
+ if (random.nextBoolean()) {
+ buffer.append(isPositive ? ":)" : ":(");
+ }
+
+ buffer.flip();
+ message.reset(buffer.array(), 0, buffer.limit(), referredTopics);
+ return message;
+ }
+ }
+
+ public static class RandomUtil {
+
+ public static Random random = new Random();
+
+ public static int[] getKFromN(int k, int n) {
+ int[] result = new int[k];
+ int cnt = 0;
+ HashSet<Integer> values = new HashSet<Integer>();
+ while (cnt < k) {
+ int val = random.nextInt(n + 1);
+ if (values.contains(val)) {
+ continue;
+ }
+
+ result[cnt++] = val;
+ values.add(val);
+ }
+ return result;
+ }
+ }
+
+ public static class FileUtil {
+
+ public static List<String> listyFile(File file) throws IOException {
+
+ BufferedReader reader = new BufferedReader(new FileReader(file));
+ String line;
+ List<String> list = new ArrayList<String>();
+ while (true) {
+ line = reader.readLine();
+ if (line == null) {
+ break;
+ }
+ list.add(line);
+ }
+ return list;
+ }
+
+ public static FileAppender getFileAppender(String filePath, boolean createIfNotExists, boolean overwrite)
+ throws IOException {
+ return new FileAppender(filePath, createIfNotExists, overwrite);
+ }
+ }
+
+ public static class FileAppender {
+
+ private final BufferedWriter writer;
+
+ public FileAppender(String filePath, boolean createIfNotExists, boolean overwrite) throws IOException {
+ File file = new File(filePath);
+ if (!file.exists()) {
+ if (createIfNotExists) {
+ new File(file.getParent()).mkdirs();
+ } else {
+ throw new IOException("path " + filePath + " does not exists");
+ }
+ }
+ this.writer = new BufferedWriter(new FileWriter(file, !overwrite));
+ }
+
+ public void appendToFile(String content) throws IOException {
+ writer.append(content);
+ writer.append("\n");
+ }
+
+ public void close() throws IOException {
+ writer.close();
+ }
+ }
+
+ public static class RandomEmploymentGenerator {
+
+ private final int percentEmployed;
+ private final Random random = new Random();
+ private final RandomDateGenerator randDateGen;
+ private final List<String> organizations;
+ private Employment emp;
+
+ public RandomEmploymentGenerator(int percentEmployed, Date beginEmpDate, Date endEmpDate, String orgListPath)
+ throws IOException {
+ this.percentEmployed = percentEmployed;
+ this.randDateGen = new RandomDateGenerator(beginEmpDate, endEmpDate);
+ organizations = FileUtil.listyFile(new File(orgListPath));
+ emp = new Employment();
+ }
+
+ public RandomEmploymentGenerator(int percentEmployed, Date beginEmpDate, Date endEmpDate, String[] orgList) {
+ this.percentEmployed = percentEmployed;
+ this.randDateGen = new RandomDateGenerator(beginEmpDate, endEmpDate);
+ organizations = new ArrayList<String>();
+ for (String org : orgList) {
+ organizations.add(org);
+ }
+ emp = new Employment();
+ }
+
+ public Employment getRandomEmployment() {
+ int empployed = random.nextInt(100) + 1;
+ boolean isEmployed = false;
+ if (empployed <= percentEmployed) {
+ isEmployed = true;
+ }
+ Date startDate = randDateGen.getNextRandomDate();
+ Date endDate = null;
+ if (!isEmployed) {
+ endDate = randDateGen.getNextRecentDate(startDate);
+ }
+ String org = organizations.get(random.nextInt(organizations.size()));
+ emp.reset(org, startDate, endDate);
+ return emp;
+ }
+ }
+
+ public static class RandomLocationGenerator {
+
+ private Random random = new Random();
+
+ private final int beginLat;
+ private final int endLat;
+ private final int beginLong;
+ private final int endLong;
+
+ private Point point;
+
+ public RandomLocationGenerator(int beginLat, int endLat, int beginLong, int endLong) {
+ this.beginLat = beginLat;
+ this.endLat = endLat;
+ this.beginLong = beginLong;
+ this.endLong = endLong;
+ this.point = new Point();
+ }
+
+ public Point getRandomPoint() {
+ int latMajor = beginLat + random.nextInt(endLat - beginLat);
+ int latMinor = random.nextInt(100);
+ float latitude = latMajor + ((float) latMinor) / 100;
+
+ int longMajor = beginLong + random.nextInt(endLong - beginLong);
+ int longMinor = random.nextInt(100);
+ float longitude = longMajor + ((float) longMinor) / 100;
+
+ point.reset(latitude, longitude);
+ return point;
+ }
+
+ }
+
+ public static class PartitionConfiguration {
+
+ private final TargetPartition targetPartition;
+ private final SourcePartition sourcePartition;
+
+ public PartitionConfiguration(SourcePartition sourcePartition, TargetPartition targetPartition) {
+ this.sourcePartition = sourcePartition;
+ this.targetPartition = targetPartition;
+ }
+
+ public TargetPartition getTargetPartition() {
+ return targetPartition;
+ }
+
+ public SourcePartition getSourcePartition() {
+ return sourcePartition;
+ }
+
+ }
+
+ public static class SourcePartition {
+
+ private final String name;
+ private final String host;
+ private final String path;
+
+ public SourcePartition(String name, String host, String path) {
+ this.name = name;
+ this.host = host;
+ this.path = path;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public String getHost() {
+ return host;
+ }
+
+ public String getPath() {
+ return path;
+ }
+ }
+
+ public static class TargetPartition {
+ private final String name;
+ private final String host;
+ private final String path;
+ private final int fbUserKeyMin;
+ private final int fbUserKeyMax;
+ private final int twUserKeyMin;
+ private final int twUserKeyMax;
+ private final int fbMessageIdMin;
+ private final int fbMessageIdMax;
+ private final int twMessageIdMin;
+ private final int twMessageIdMax;
+ private final int commonUsers;
+
+ public TargetPartition(String partitionName, String host, String path, int fbUserKeyMin, int fbUserKeyMax,
+ int twUserKeyMin, int twUserKeyMax, int fbMessageIdMin, int fbMessageIdMax, int twMessageIdMin,
+ int twMessageIdMax, int commonUsers) {
+ this.name = partitionName;
+ this.host = host;
+ this.path = path;
+ this.fbUserKeyMin = fbUserKeyMin;
+ this.fbUserKeyMax = fbUserKeyMax;
+ this.twUserKeyMin = twUserKeyMin;
+ this.twUserKeyMax = twUserKeyMax;
+ this.twMessageIdMin = twMessageIdMin;
+ this.twMessageIdMax = twMessageIdMax;
+ this.fbMessageIdMin = fbMessageIdMin;
+ this.fbMessageIdMax = fbMessageIdMax;
+ this.commonUsers = commonUsers;
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append(name);
+ builder.append(" ");
+ builder.append(host);
+ builder.append("\n");
+ builder.append(path);
+ builder.append("\n");
+ builder.append("fbUser:key:min");
+ builder.append(fbUserKeyMin);
+
+ builder.append("\n");
+ builder.append("fbUser:key:max");
+ builder.append(fbUserKeyMax);
+
+ builder.append("\n");
+ builder.append("twUser:key:min");
+ builder.append(twUserKeyMin);
+
+ builder.append("\n");
+ builder.append("twUser:key:max");
+ builder.append(twUserKeyMax);
+
+ builder.append("\n");
+ builder.append("fbMessage:key:min");
+ builder.append(fbMessageIdMin);
+
+ builder.append("\n");
+ builder.append("fbMessage:key:max");
+ builder.append(fbMessageIdMax);
+
+ builder.append("\n");
+ builder.append("twMessage:key:min");
+ builder.append(twMessageIdMin);
+
+ builder.append("\n");
+ builder.append("twMessage:key:max");
+ builder.append(twMessageIdMax);
+
+ builder.append("\n");
+ builder.append("twMessage:key:max");
+ builder.append(twMessageIdMax);
+
+ builder.append("\n");
+ builder.append("commonUsers");
+ builder.append(commonUsers);
+
+ return new String(builder);
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public String getHost() {
+ return host;
+ }
+
+ public int getFbUserKeyMin() {
+ return fbUserKeyMin;
+ }
+
+ public int getFbUserKeyMax() {
+ return fbUserKeyMax;
+ }
+
+ public int getTwUserKeyMin() {
+ return twUserKeyMin;
+ }
+
+ public int getTwUserKeyMax() {
+ return twUserKeyMax;
+ }
+
+ public int getFbMessageIdMin() {
+ return fbMessageIdMin;
+ }
+
+ public int getFbMessageIdMax() {
+ return fbMessageIdMax;
+ }
+
+ public int getTwMessageIdMin() {
+ return twMessageIdMin;
+ }
+
+ public int getTwMessageIdMax() {
+ return twMessageIdMax;
+ }
+
+ public int getCommonUsers() {
+ return commonUsers;
+ }
+
+ public String getPath() {
+ return path;
+ }
+ }
+
+ public static class Employment {
+
+ private String organization;
+ private Date startDate;
+ private Date endDate;
+
+ public Employment(String organization, Date startDate, Date endDate) {
+ this.organization = organization;
+ this.startDate = startDate;
+ this.endDate = endDate;
+ }
+
+ public Employment() {
+ }
+
+ public void reset(String organization, Date startDate, Date endDate) {
+ this.organization = organization;
+ this.startDate = startDate;
+ this.endDate = endDate;
+ }
+
+ public String getOrganization() {
+ return organization;
+ }
+
+ public Date getStartDate() {
+ return startDate;
+ }
+
+ public Date getEndDate() {
+ return endDate;
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder("");
+ builder.append("{");
+ builder.append("\"organization-name\":");
+ builder.append("\"" + organization + "\"");
+ builder.append(",");
+ builder.append("\"start-date\":");
+ builder.append(startDate);
+ if (endDate != null) {
+ builder.append(",");
+ builder.append("\"end-date\":");
+ builder.append(endDate);
+ }
+ builder.append("}");
+ return new String(builder);
+ }
+
+ }
+
+ public static class FacebookMessage {
+
+ private int messageId;
+ private int authorId;
+ private int inResponseTo;
+ private Point senderLocation;
+ private Message message;
+
+ public int getMessageId() {
+ return messageId;
+ }
+
+ public int getAuthorID() {
+ return authorId;
+ }
+
+ public Point getSenderLocation() {
+ return senderLocation;
+ }
+
+ public Message getMessage() {
+ return message;
+ }
+
+ public int getInResponseTo() {
+ return inResponseTo;
+ }
+
+ public FacebookMessage() {
+
+ }
+
+ public FacebookMessage(int messageId, int authorId, int inResponseTo, Point senderLocation, Message message) {
+ this.messageId = messageId;
+ this.authorId = authorId;
+ this.inResponseTo = inResponseTo;
+ this.senderLocation = senderLocation;
+ this.message = message;
+ }
+
+ public void reset(int messageId, int authorId, int inResponseTo, Point senderLocation, Message message) {
+ this.messageId = messageId;
+ this.authorId = authorId;
+ this.inResponseTo = inResponseTo;
+ this.senderLocation = senderLocation;
+ this.message = message;
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("{");
+ builder.append("\"message-id\":");
+ builder.append(messageId);
+ builder.append(",");
+ builder.append("\"author-id\":");
+ builder.append(authorId);
+ builder.append(",");
+ builder.append("\"in-response-to\":");
+ builder.append(inResponseTo);
+ builder.append(",");
+ builder.append("\"sender-location\":");
+ builder.append(senderLocation);
+ builder.append(",");
+ builder.append("\"message\":");
+ builder.append("\"");
+ for (int i = 0; i < message.getLength(); i++) {
+ builder.append(message.charAt(i));
+ }
+ builder.append("\"");
+ builder.append("}");
+ return new String(builder);
+ }
+ }
+
+ public static class FacebookUser {
+
+ private int id;
+ private String alias;
+ private String name;
+ private String userSince;
+ private int[] friendIds;
+ private Employment employment;
+
+ public FacebookUser() {
+
+ }
+
+ public FacebookUser(int id, String alias, String name, String userSince, int[] friendIds, Employment employment) {
+ this.id = id;
+ this.alias = alias;
+ this.name = name;
+ this.userSince = userSince;
+ this.friendIds = friendIds;
+ this.employment = employment;
+ }
+
+ public int getId() {
+ return id;
+ }
+
+ public String getAlias() {
+ return alias;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public String getUserSince() {
+ return userSince;
+ }
+
+ public int[] getFriendIds() {
+ return friendIds;
+ }
+
+ public Employment getEmployment() {
+ return employment;
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("{");
+ builder.append("\"id\":" + id);
+ builder.append(",");
+ builder.append("\"alias\":" + "\"" + alias + "\"");
+ builder.append(",");
+ builder.append("\"name\":" + "\"" + name + "\"");
+ builder.append(",");
+ builder.append("\"user-since\":" + userSince);
+ builder.append(",");
+ builder.append("\"friend-ids\":");
+ builder.append("{{");
+ for (int i = 0; i < friendIds.length; i++) {
+ builder.append(friendIds[i]);
+ builder.append(",");
+ }
+ if (friendIds.length > 0) {
+ builder.deleteCharAt(builder.lastIndexOf(","));
+ }
+ builder.append("}}");
+ builder.append(",");
+ builder.append("\"employment\":");
+ builder.append("[");
+ builder.append(employment.toString());
+ builder.append("]");
+ builder.append("}");
+ return builder.toString();
+ }
+
+ public void setId(int id) {
+ this.id = id;
+ }
+
+ public void setAlias(String alias) {
+ this.alias = alias;
+ }
+
+ public void setName(String name) {
+ this.name = name;
+ }
+
+ public void setUserSince(String userSince) {
+ this.userSince = userSince;
+ }
+
+ public void setFriendIds(int[] friendIds) {
+ this.friendIds = friendIds;
+ }
+
+ public void setEmployment(Employment employment) {
+ this.employment = employment;
+ }
+
+ public void reset(int id, String alias, String name, String userSince, int[] friendIds, Employment employment) {
+ this.id = id;
+ this.alias = alias;
+ this.name = name;
+ this.userSince = userSince;
+ this.friendIds = friendIds;
+ this.employment = employment;
+ }
+ }
+
+ public static class TweetMessage {
+
+ private String tweetid;
+ private TwitterUser user;
+ private Point senderLocation;
+ private DateTime sendTime;
+ private List<String> referredTopics;
+ private Message messageText;
+
+ public TweetMessage() {
+
+ }
+
+ public TweetMessage(String tweetid, TwitterUser user, Point senderLocation, DateTime sendTime,
+ List<String> referredTopics, Message messageText) {
+ this.tweetid = tweetid;
+ this.user = user;
+ this.senderLocation = senderLocation;
+ this.sendTime = sendTime;
+ this.referredTopics = referredTopics;
+ this.messageText = messageText;
+ }
+
+ public void reset(String tweetid, TwitterUser user, Point senderLocation, DateTime sendTime,
+ List<String> referredTopics, Message messageText) {
+ this.tweetid = tweetid;
+ this.user = user;
+ this.senderLocation = senderLocation;
+ this.sendTime = sendTime;
+ this.referredTopics = referredTopics;
+ this.messageText = messageText;
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("{");
+ builder.append("\"tweetid\":");
+ builder.append("\"" + tweetid + "\"");
+ builder.append(",");
+ builder.append("\"user\":");
+ builder.append(user);
+ builder.append(",");
+ builder.append("\"sender-location\":");
+ builder.append(senderLocation);
+ builder.append(",");
+ builder.append("\"send-time\":");
+ builder.append(sendTime);
+ builder.append(",");
+ builder.append("\"referred-topics\":");
+ builder.append("{{");
+ for (String topic : referredTopics) {
+ builder.append("\"" + topic + "\"");
+ builder.append(",");
+ }
+ if (referredTopics.size() > 0) {
+ builder.deleteCharAt(builder.lastIndexOf(","));
+ }
+ builder.append("}}");
+ builder.append(",");
+ builder.append("\"message-text\":");
+ builder.append("\"");
+ for (int i = 0; i < messageText.getLength(); i++) {
+ builder.append(messageText.charAt(i));
+ }
+ builder.append("\"");
+ builder.append("}");
+ return new String(builder);
+ }
+
+ public String getTweetid() {
+ return tweetid;
+ }
+
+ public void setTweetid(String tweetid) {
+ this.tweetid = tweetid;
+ }
+
+ public TwitterUser getUser() {
+ return user;
+ }
+
+ public void setUser(TwitterUser user) {
+ this.user = user;
+ }
+
+ public Point getSenderLocation() {
+ return senderLocation;
+ }
+
+ public void setSenderLocation(Point senderLocation) {
+ this.senderLocation = senderLocation;
+ }
+
+ public DateTime getSendTime() {
+ return sendTime;
+ }
+
+ public void setSendTime(DateTime sendTime) {
+ this.sendTime = sendTime;
+ }
+
+ public List<String> getReferredTopics() {
+ return referredTopics;
+ }
+
+ public void setReferredTopics(List<String> referredTopics) {
+ this.referredTopics = referredTopics;
+ }
+
+ public Message getMessageText() {
+ return messageText;
+ }
+
+ public void setMessageText(Message messageText) {
+ this.messageText = messageText;
+ }
+
+ }
+
+ public static class TwitterUser {
+
+ private String screenName;
+ private String lang = "en";
+ private int friendsCount;
+ private int statusesCount;
+ private String name;
+ private int followersCount;
+
+ public TwitterUser() {
+
+ }
+
+ public TwitterUser(String screenName, int friendsCount, int statusesCount, String name, int followersCount) {
+ this.screenName = screenName;
+ this.friendsCount = friendsCount;
+ this.statusesCount = statusesCount;
+ this.name = name;
+ this.followersCount = followersCount;
+ }
+
+ public void reset(String screenName, int friendsCount, int statusesCount, String name, int followersCount) {
+ this.screenName = screenName;
+ this.friendsCount = friendsCount;
+ this.statusesCount = statusesCount;
+ this.name = name;
+ this.followersCount = followersCount;
+ }
+
+ public String getScreenName() {
+ return screenName;
+ }
+
+ public int getFriendsCount() {
+ return friendsCount;
+ }
+
+ public int getStatusesCount() {
+ return statusesCount;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public int getFollowersCount() {
+ return followersCount;
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("{");
+ builder.append("\"screen-name\":" + "\"" + screenName + "\"");
+ builder.append(",");
+ builder.append("\"lang\":" + "\"" + lang + "\"");
+ builder.append(",");
+ builder.append("\"friends_count\":" + friendsCount);
+ builder.append(",");
+ builder.append("\"statuses_count\":" + statusesCount);
+ builder.append(",");
+ builder.append("\"name\":" + "\"" + name + "\"");
+ builder.append(",");
+ builder.append("\"followers_count\":" + followersCount);
+ builder.append("}");
+ return builder.toString();
+ }
+
+ }
+
+ public static class DistributionHandler {
+
+ private final ZipfGenerator zipfGen;
+ private final int totalUsers;
+ private final int totalMessages;
+ private Random random = new Random();
+
+ public DistributionHandler(int totalMessages, double skew, int totalNumUsers) {
+ zipfGen = new ZipfGenerator(totalMessages, skew);
+ totalUsers = totalNumUsers;
+ this.totalMessages = totalMessages;
+ }
+
+ public int getFromDistribution(int rank) {
+ double prob = zipfGen.getProbability(rank);
+ int numMessages = (int) (prob * totalMessages);
+ return numMessages;
+ }
+
+ public static void main(String args[]) {
+ int totalMessages = 1000 * 4070;
+ double skew = 0.5;
+ int totalUsers = 4070;
+ DistributionHandler d = new DistributionHandler(totalMessages, skew, totalUsers);
+ int sum = 0;
+ for (int i = totalUsers; i >= 1; i--) {
+ float contrib = d.getFromDistribution(i);
+ sum += contrib;
+ System.out.println(i + ":" + contrib);
+ }
+
+ System.out.println("SUM" + ":" + sum);
+
+ }
+ }
+
+ public static class ZipfGenerator {
+
+ private Random rnd = new Random(System.currentTimeMillis());
+ private int size;
+ private double skew;
+ private double bottom = 0;
+
+ public ZipfGenerator(int size, double skew) {
+ this.size = size;
+ this.skew = skew;
+ for (int i = 1; i < size; i++) {
+ this.bottom += (1 / Math.pow(i, this.skew));
+ }
+ }
+
+ // the next() method returns an rank id. The frequency of returned rank
+ // ids are follows Zipf distribution.
+ public int next() {
+ int rank;
+ double friquency = 0;
+ double dice;
+ rank = rnd.nextInt(size);
+ friquency = (1.0d / Math.pow(rank, this.skew)) / this.bottom;
+ dice = rnd.nextDouble();
+ while (!(dice < friquency)) {
+ rank = rnd.nextInt(size);
+ friquency = (1.0d / Math.pow(rank, this.skew)) / this.bottom;
+ dice = rnd.nextDouble();
+ }
+ return rank;
+ }
+
+ // This method returns a probability that the given rank occurs.
+ public double getProbability(int rank) {
+ return (1.0d / Math.pow(rank, this.skew)) / this.bottom;
+ }
+
+ public static void main(String[] args) throws IOException {
+ int total = (int) (3.7 * 1000 * 1000);
+ int skew = 2;
+ int numUsers = 1000 * 1000;
+ /*
+ * if (args.length != 2) { System.out.println("usage:" +
+ * "./zipf size skew"); System.exit(-1); }
+ */
+ BufferedWriter buf = new BufferedWriter(new FileWriter(new File("/tmp/zip_output")));
+ ZipfGenerator zipf = new ZipfGenerator(total, skew);
+ double sum = 0;
+ for (int i = 1; i <= numUsers; i++) {
+ double prob = zipf.getProbability(i);
+ double contribution = (double) (prob * total);
+ String contrib = i + ":" + contribution;
+ buf.write(contrib);
+ buf.write("\n");
+ System.out.println(contrib);
+ sum += contribution;
+ }
+ System.out.println("sum is :" + sum);
+ }
+ }
+
+ public static class PartitionElement implements ILibraryElement {
+ private final String name;
+ private final String host;
+ private final int fbUserKeyMin;
+ private final int fbUserKeyMax;
+ private final int twUserKeyMin;
+ private final int twUserKeyMax;
+ private final int fbMessageIdMin;
+ private final int fbMessageIdMax;
+ private final int twMessageIdMin;
+ private final int twMessageIdMax;
+
+ public PartitionElement(String partitionName, String host, int fbUserKeyMin, int fbUserKeyMax,
+ int twUserKeyMin, int twUserKeyMax, int fbMessageIdMin, int fbMessageIdMax, int twMessageIdMin,
+ int twMessageIdMax) {
+ this.name = partitionName;
+ this.host = host;
+ this.fbUserKeyMin = fbUserKeyMin;
+ this.fbUserKeyMax = fbUserKeyMax;
+ this.twUserKeyMin = twUserKeyMax;
+ this.twUserKeyMax = twUserKeyMax;
+ this.twMessageIdMin = twMessageIdMin;
+ this.twMessageIdMax = twMessageIdMax;
+ this.fbMessageIdMin = fbMessageIdMin;
+ this.fbMessageIdMax = fbMessageIdMax;
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append(name);
+ builder.append(" ");
+ builder.append(host);
+ builder.append("\n");
+ builder.append("fbUser:key:min");
+ builder.append(fbUserKeyMin);
+
+ builder.append("\n");
+ builder.append("fbUser:key:max");
+ builder.append(fbUserKeyMax);
+
+ builder.append("\n");
+ builder.append("twUser:key:min");
+ builder.append(twUserKeyMin);
+
+ builder.append("\n");
+ builder.append("twUser:key:max");
+ builder.append(twUserKeyMax);
+
+ builder.append("\n");
+ builder.append("fbMessage:key:min");
+ builder.append(fbMessageIdMin);
+
+ builder.append("\n");
+ builder.append("fbMessage:key:max");
+ builder.append(fbMessageIdMax);
+
+ builder.append("\n");
+ builder.append("twMessage:key:min");
+ builder.append(twMessageIdMin);
+
+ builder.append("\n");
+ builder.append("twMessage:key:max");
+ builder.append(twMessageIdMax);
+
+ builder.append("\n");
+ builder.append("twMessage:key:max");
+ builder.append(twUserKeyMin);
+
+ return new String(builder);
+ }
+
+ @Override
+ public String getName() {
+ return "Partition";
+ }
+
+ }
+
+ interface ILibraryElement {
+
+ public enum ElementType {
+ PARTITION
+ }
+
+ public String getName();
+
+ }
+
+ public static class Configuration {
+
+ private final float numMB;
+ private final String unit;
+
+ private final List<SourcePartition> sourcePartitions;
+ private List<TargetPartition> targetPartitions;
+
+ public Configuration(float numMB, String unit, List<SourcePartition> partitions) throws IOException {
+ this.numMB = numMB;
+ this.unit = unit;
+ this.sourcePartitions = partitions;
+
+ }
+
+ public float getNumMB() {
+ return numMB;
+ }
+
+ public String getUnit() {
+ return unit;
+ }
+
+ public List<SourcePartition> getSourcePartitions() {
+ return sourcePartitions;
+ }
+
+ public List<TargetPartition> getTargetPartitions() {
+ return targetPartitions;
+ }
+
+ public void setTargetPartitions(List<TargetPartition> targetPartitions) {
+ this.targetPartitions = targetPartitions;
+ }
+
+ }
+
+ public static class XMLUtil {
+
+ public static void writeToXML(Configuration conf, String filePath) throws IOException,
+ ParserConfigurationException, TransformerException {
+
+ DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance();
+ DocumentBuilder docBuilder = docFactory.newDocumentBuilder();
+
+ // root elements
+ Document doc = docBuilder.newDocument();
+ Element rootElement = doc.createElement("Partitions");
+ doc.appendChild(rootElement);
+
+ int index = 0;
+ for (TargetPartition partition : conf.getTargetPartitions()) {
+ writePartitionElement(conf.getSourcePartitions().get(index), partition, rootElement, doc);
+ }
+
+ TransformerFactory transformerFactory = TransformerFactory.newInstance();
+ Transformer transformer = transformerFactory.newTransformer();
+
+ transformer.setOutputProperty(OutputKeys.ENCODING, "utf-8");
+ transformer.setOutputProperty(OutputKeys.OMIT_XML_DECLARATION, "no");
+ transformer.setOutputProperty(OutputKeys.INDENT, "yes");
+ transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "2");
+
+ DOMSource source = new DOMSource(doc);
+ StreamResult result = new StreamResult(new File(filePath));
+
+ transformer.transform(source, result);
+
+ }
+
+ public static void writePartitionInfo(Configuration conf, String filePath) throws IOException {
+ BufferedWriter bw = new BufferedWriter(new FileWriter(filePath));
+ for (SourcePartition sp : conf.getSourcePartitions()) {
+ bw.write(sp.getHost() + ":" + sp.getName() + ":" + sp.getPath());
+ bw.write("\n");
+ }
+ bw.close();
+ }
+
+ public static Document getDocument(String filePath) throws Exception {
+ File inputFile = new File(filePath);
+ DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+ DocumentBuilder db = dbf.newDocumentBuilder();
+ Document doc = db.parse(inputFile);
+ doc.getDocumentElement().normalize();
+ return doc;
+ }
+
+ public static Configuration getConfiguration(String filePath) throws Exception {
+ Configuration conf = getConfiguration(getDocument(filePath));
+ PartitionMetrics metrics = new PartitionMetrics(conf.getNumMB(), conf.getUnit(), conf.getSourcePartitions()
+ .size());
+ List<TargetPartition> targetPartitions = getTargetPartitions(metrics, conf.getSourcePartitions());
+ conf.setTargetPartitions(targetPartitions);
+ return conf;
+ }
+
+ public static Configuration getConfiguration(Document document) throws IOException {
+ Element rootEle = document.getDocumentElement();
+ NodeList nodeList = rootEle.getChildNodes();
+ float size = Float.parseFloat(getStringValue((Element) nodeList, "size"));
+ String unit = getStringValue((Element) nodeList, "unit");
+ List<SourcePartition> sourcePartitions = getSourcePartitions(document);
+ return new Configuration(size, unit, sourcePartitions);
+ }
+
+ public static List<SourcePartition> getSourcePartitions(Document document) {
+ Element rootEle = document.getDocumentElement();
+ NodeList nodeList = rootEle.getElementsByTagName("partition");
+ List<SourcePartition> sourcePartitions = new ArrayList<SourcePartition>();
+ for (int i = 0; i < nodeList.getLength(); i++) {
+ Node node = nodeList.item(i);
+ sourcePartitions.add(getSourcePartition((Element) node));
+ }
+ return sourcePartitions;
+ }
+
+ public static SourcePartition getSourcePartition(Element functionElement) {
+ String name = getStringValue(functionElement, "name");
+ String host = getStringValue(functionElement, "host");
+ String path = getStringValue(functionElement, "path");
+ SourcePartition sp = new SourcePartition(name, host, path);
+ return sp;
+ }
+
+ public static String getStringValue(Element element, String tagName) {
+ String textValue = null;
+ NodeList nl = element.getElementsByTagName(tagName);
+ if (nl != null && nl.getLength() > 0) {
+ Element el = (Element) nl.item(0);
+ textValue = el.getFirstChild().getNodeValue();
+ }
+ return textValue;
+ }
+
+ public static PartitionConfiguration getPartitionConfiguration(String filePath, String partitionName)
+ throws Exception {
+ PartitionConfiguration pconf = getPartitionConfigurations(getDocument(filePath), partitionName);
+ return pconf;
+ }
+
+ public static PartitionConfiguration getPartitionConfigurations(Document document, String partitionName)
+ throws IOException {
+
+ Element rootEle = document.getDocumentElement();
+ NodeList nodeList = rootEle.getElementsByTagName("Partition");
+
+ for (int i = 0; i < nodeList.getLength(); i++) {
+ Node node = nodeList.item(i);
+ Element nodeElement = (Element) node;
+ String name = getStringValue(nodeElement, "name");
+ if (!name.equalsIgnoreCase(partitionName)) {
+ continue;
+ }
+ String host = getStringValue(nodeElement, "host");
+ String path = getStringValue(nodeElement, "path");
+
+ String fbUserKeyMin = getStringValue(nodeElement, "fbUserKeyMin");
+ String fbUserKeyMax = getStringValue(nodeElement, "fbUserKeyMax");
+ String twUserKeyMin = getStringValue(nodeElement, "twUserKeyMin");
+ String twUserKeyMax = getStringValue(nodeElement, "twUserKeyMax");
+ String fbMessageKeyMin = getStringValue(nodeElement, "fbMessageKeyMin");
+
+ String fbMessageKeyMax = getStringValue(nodeElement, "fbMessageKeyMax");
+ String twMessageKeyMin = getStringValue(nodeElement, "twMessageKeyMin");
+ String twMessageKeyMax = getStringValue(nodeElement, "twMessageKeyMax");
+ String numCommonUsers = getStringValue(nodeElement, "numCommonUsers");
+
+ SourcePartition sp = new SourcePartition(name, host, path);
+
+ TargetPartition tp = new TargetPartition(partitionName, host, path, Integer.parseInt(fbUserKeyMin),
+ Integer.parseInt(fbUserKeyMax), Integer.parseInt(twUserKeyMin), Integer.parseInt(twUserKeyMax),
+ Integer.parseInt(fbMessageKeyMin), Integer.parseInt(fbMessageKeyMax),
+ Integer.parseInt(twMessageKeyMin), Integer.parseInt(twMessageKeyMax),
+ Integer.parseInt(numCommonUsers));
+ PartitionConfiguration pc = new PartitionConfiguration(sp, tp);
+ return pc;
+ }
+ return null;
+ }
+
+ public static List<TargetPartition> getTargetPartitions(PartitionMetrics metrics,
+ List<SourcePartition> sourcePartitions) {
+ List<TargetPartition> partitions = new ArrayList<TargetPartition>();
+ int fbUserKeyMin = 1;
+ int twUserKeyMin = 1;
+ int fbMessageIdMin = 1;
+ int twMessageIdMin = 1;
+
+ for (SourcePartition sp : sourcePartitions) {
+ int fbUserKeyMax = fbUserKeyMin + metrics.getFbOnlyUsers() + metrics.getCommonUsers() - 1;
+ int twUserKeyMax = twUserKeyMin + metrics.getTwitterOnlyUsers() + metrics.getCommonUsers() - 1;
+
+ int fbMessageIdMax = fbMessageIdMin + metrics.getFbMessages() - 1;
+ int twMessageIdMax = twMessageIdMin + metrics.getTwMessages() - 1;
+ TargetPartition pe = new TargetPartition(sp.getName(), sp.getHost(), sp.getPath(), fbUserKeyMin,
+ fbUserKeyMax, twUserKeyMin, twUserKeyMax, fbMessageIdMin, fbMessageIdMax, twMessageIdMin,
+ twMessageIdMax, metrics.getCommonUsers());
+ partitions.add(pe);
+
+ fbUserKeyMin = fbUserKeyMax + 1;
+ twUserKeyMin = twUserKeyMax + 1;
+
+ fbMessageIdMin = fbMessageIdMax + 1;
+ twMessageIdMin = twMessageIdMax + 1;
+ }
+
+ return partitions;
+ }
+
+ public static void writePartitionElement(SourcePartition sourcePartition, TargetPartition partition,
+ Element rootElement, Document doc) {
+ // staff elements
+ Element pe = doc.createElement("Partition");
+ rootElement.appendChild(pe);
+
+ // name element
+ Element name = doc.createElement("name");
+ name.appendChild(doc.createTextNode("" + partition.getName()));
+ pe.appendChild(name);
+
+ // host element
+ Element host = doc.createElement("host");
+ host.appendChild(doc.createTextNode("" + partition.getHost()));
+ pe.appendChild(host);
+
+ // path element
+ Element path = doc.createElement("path");
+ path.appendChild(doc.createTextNode("" + partition.getPath()));
+ pe.appendChild(path);
+
+ // fbUserKeyMin element
+ Element fbUserKeyMin = doc.createElement("fbUserKeyMin");
+ fbUserKeyMin.appendChild(doc.createTextNode("" + partition.getFbUserKeyMin()));
+ pe.appendChild(fbUserKeyMin);
+
+ // fbUserKeyMax element
+ Element fbUserKeyMax = doc.createElement("fbUserKeyMax");
+ fbUserKeyMax.appendChild(doc.createTextNode("" + partition.getFbUserKeyMax()));
+ pe.appendChild(fbUserKeyMax);
+
+ // twUserKeyMin element
+ Element twUserKeyMin = doc.createElement("twUserKeyMin");
+ twUserKeyMin.appendChild(doc.createTextNode("" + partition.getTwUserKeyMin()));
+ pe.appendChild(twUserKeyMin);
+
+ // twUserKeyMax element
+ Element twUserKeyMax = doc.createElement("twUserKeyMax");
+ twUserKeyMax.appendChild(doc.createTextNode("" + partition.getTwUserKeyMax()));
+ pe.appendChild(twUserKeyMax);
+
+ // fbMessgeKeyMin element
+ Element fbMessageKeyMin = doc.createElement("fbMessageKeyMin");
+ fbMessageKeyMin.appendChild(doc.createTextNode("" + partition.getFbMessageIdMin()));
+ pe.appendChild(fbMessageKeyMin);
+
+ // fbMessgeKeyMin element
+ Element fbMessageKeyMax = doc.createElement("fbMessageKeyMax");
+ fbMessageKeyMax.appendChild(doc.createTextNode("" + partition.getFbMessageIdMax()));
+ pe.appendChild(fbMessageKeyMax);
+
+ // twMessgeKeyMin element
+ Element twMessageKeyMin = doc.createElement("twMessageKeyMin");
+ twMessageKeyMin.appendChild(doc.createTextNode("" + partition.getTwMessageIdMin()));
+ pe.appendChild(twMessageKeyMin);
+
+ // twMessgeKeyMin element
+ Element twMessageKeyMax = doc.createElement("twMessageKeyMax");
+ twMessageKeyMax.appendChild(doc.createTextNode("" + partition.getTwMessageIdMax()));
+ pe.appendChild(twMessageKeyMax);
+
+ // twMessgeKeyMin element
+ Element numCommonUsers = doc.createElement("numCommonUsers");
+ numCommonUsers.appendChild(doc.createTextNode("" + partition.getCommonUsers()));
+ pe.appendChild(numCommonUsers);
+
+ }
+
+ public static void main(String args[]) throws Exception {
+ String confFile = "/Users/rgrove1/work/research/asterix/icde/data-gen/conf/conf.xml";
+ String outputPath = "/Users/rgrove1/work/research/asterix/icde/data-gen/output/conf-output.xml";
+ Configuration conf = getConfiguration(confFile);
+ writeToXML(conf, outputPath);
+ }
+
+ }
+
+ public static class Date {
+
+ private int day;
+ private int month;
+ private int year;
+
+ public Date(int month, int day, int year) {
+ this.month = month;
+ this.day = day;
+ this.year = year;
+ }
+
+ public void reset(int month, int day, int year) {
+ this.month = month;
+ this.day = day;
+ this.year = year;
+ }
+
+ public int getDay() {
+ return day;
+ }
+
+ public int getMonth() {
+ return month;
+ }
+
+ public int getYear() {
+ return year;
+ }
+
+ public Date() {
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("date");
+ builder.append("(\"");
+ builder.append(year);
+ builder.append("-");
+ builder.append(month < 10 ? "0" + month : "" + month);
+ builder.append("-");
+ builder.append(day < 10 ? "0" + day : "" + day);
+ builder.append("\")");
+ return builder.toString();
+ }
+
+ public void setDay(int day) {
+ this.day = day;
+ }
+
+ public void setMonth(int month) {
+ this.month = month;
+ }
+
+ public void setYear(int year) {
+ this.year = year;
+ }
+ }
+
+ public static class PartitionMetrics {
+
+ private final int fbMessages;
+ private final int twMessages;
+
+ private final int fbOnlyUsers;
+ private final int twitterOnlyUsers;
+ private final int commonUsers;
+
+ public PartitionMetrics(float number, String unit, int numPartitions) throws IOException {
+
+ int factor = 0;
+ if (unit.equalsIgnoreCase("MB")) {
+ factor = 1024 * 1024;
+ } else if (unit.equalsIgnoreCase("GB")) {
+ factor = 1024 * 1024 * 1024;
+ } else if (unit.equalsIgnoreCase("TB")) {
+ factor = 1024 * 1024 * 1024 * 1024;
+ } else
+ throw new IOException("Invalid unit:" + unit);
+
+ fbMessages = (int) (((number * factor * 0.80) / 258) / numPartitions);
+ twMessages = (int) (fbMessages * 1.1 / 0.35);
+
+ fbOnlyUsers = (int) ((number * factor * 0.20 * 0.0043)) / numPartitions;
+ twitterOnlyUsers = (int) (0.25 * fbOnlyUsers);
+ commonUsers = (int) (0.1 * fbOnlyUsers);
+ }
+
+ public int getFbMessages() {
+ return fbMessages;
+ }
+
+ public int getTwMessages() {
+ return twMessages;
+ }
+
+ public int getFbOnlyUsers() {
+ return fbOnlyUsers;
+ }
+
+ public int getTwitterOnlyUsers() {
+ return twitterOnlyUsers;
+ }
+
+ public int getCommonUsers() {
+ return commonUsers;
+ }
+
+ }
+
+ public static String[] lastNames = { "Hoopengarner", "Harrow", "Gardner", "Blyant", "Best", "Buttermore", "Gronko",
+ "Mayers", "Countryman", "Neely", "Ruhl", "Taggart", "Bash", "Cason", "Hil", "Zalack", "Mingle", "Carr",
+ "Rohtin", "Wardle", "Pullman", "Wire", "Kellogg", "Hiles", "Keppel", "Bratton", "Sutton", "Wickes",
+ "Muller", "Friedline", "Llora", "Elizabeth", "Anderson", "Gaskins", "Rifler", "Vinsant", "Stanfield",
+ "Black", "Guest", "Hujsak", "Carter", "Weidemann", "Hays", "Patton", "Hayhurst", "Paynter", "Cressman",
+ "Fiddler", "Evans", "Sherlock", "Woodworth", "Jackson", "Bloise", "Schneider", "Ring", "Kepplinger",
+ "James", "Moon", "Bennett", "Bashline", "Ryals", "Zeal", "Christman", "Milliron", "Nash", "Ewing", "Camp",
+ "Mason", "Richardson", "Bowchiew", "Hahn", "Wilson", "Wood", "Toyley", "Williamson", "Lafortune", "Errett",
+ "Saltser", "Hirleman", "Brindle", "Newbiggin", "Ulery", "Lambert", "Shick", "Kuster", "Moore", "Finck",
+ "Powell", "Jolce", "Townsend", "Sauter", "Cowher", "Wolfe", "Cavalet", "Porter", "Laborde", "Ballou",
+ "Murray", "Stoddard", "Pycroft", "Milne", "King", "Todd", "Staymates", "Hall", "Romanoff", "Keilbach",
+ "Sandford", "Hamilton", "Fye", "Kline", "Weeks", "Mcelroy", "Mccullough", "Bryant", "Hill", "Moore",
+ "Ledgerwood", "Prevatt", "Eckert", "Read", "Hastings", "Doverspike", "Allshouse", "Bryan", "Mccallum",
+ "Lombardi", "Mckendrick", "Cattley", "Barkley", "Steiner", "Finlay", "Priebe", "Armitage", "Hall", "Elder",
+ "Erskine", "Hatcher", "Walker", "Pearsall", "Dunkle", "Haile", "Adams", "Miller", "Newbern", "Basinger",
+ "Fuhrer", "Brinigh", "Mench", "Blackburn", "Bastion", "Mccune", "Bridger", "Hynes", "Quinn", "Courtney",
+ "Geddinge", "Field", "Seelig", "Cable", "Earhart", "Harshman", "Roby", "Beals", "Berry", "Reed", "Hector",
+ "Pittman", "Haverrman", "Kalp", "Briner", "Joghs", "Cowart", "Close", "Wynne", "Harden", "Weldy",
+ "Stephenson", "Hildyard", "Moberly", "Wells", "Mackendoerfer", "Fisher", "Oppie", "Oneal", "Churchill",
+ "Keister", "Alice", "Tavoularis", "Fisher", "Hair", "Burns", "Veith", "Wile", "Fuller", "Fields", "Clark",
+ "Randolph", "Stone", "Mcclymonds", "Holtzer", "Donkin", "Wilkinson", "Rosensteel", "Albright", "Stahl",
+ "Fox", "Kadel", "Houser", "Hanseu", "Henderson", "Davis", "Bicknell", "Swain", "Mercer", "Holdeman",
+ "Enderly", "Caesar", "Margaret", "Munshower", "Elless", "Lucy", "Feufer", "Schofield", "Graham",
+ "Blatenberger", "Benford", "Akers", "Campbell", "Ann", "Sadley", "Ling", "Gongaware", "Schmidt", "Endsley",
+ "Groah", "Flanders", "Reichard", "Lowstetter", "Sandblom", "Griffis", "Basmanoff", "Coveney", "Hawker",
+ "Archibald", "Hutton", "Barnes", "Diegel", "Raybould", "Focell", "Breitenstein", "Murray", "Chauvin",
+ "Busk", "Pheleps", "Teagarden", "Northey", "Baumgartner", "Fleming", "Harris", "Parkinson", "Carpenter",
+ "Whirlow", "Bonner", "Wortman", "Rogers", "Scott", "Lowe", "Mckee", "Huston", "Bullard", "Throckmorton",
+ "Rummel", "Mathews", "Dull", "Saline", "Tue", "Woolery", "Lalty", "Schrader", "Ramsey", "Eisenmann",
+ "Philbrick", "Sybilla", "Wallace", "Fonblanque", "Paul", "Orbell", "Higgens", "Casteel", "Franks",
+ "Demuth", "Eisenman", "Hay", "Robinson", "Fischer", "Hincken", "Wylie", "Leichter", "Bousum",
+ "Littlefield", "Mcdonald", "Greif", "Rhodes", "Wall", "Steele", "Baldwin", "Smith", "Stewart", "Schere",
+ "Mary", "Aultman", "Emrick", "Guess", "Mitchell", "Painter", "Aft", "Hasely", "Weldi", "Loewentsein",
+ "Poorbaugh", "Kepple", "Noton", "Judge", "Jackson", "Style", "Adcock", "Diller", "Marriman", "Johnston",
+ "Children", "Monahan", "Ehret", "Shaw", "Congdon", "Pinney", "Millard", "Crissman", "Tanner", "Rose",
+ "Knisely", "Cypret", "Sommer", "Poehl", "Hardie", "Bender", "Overholt", "Gottwine", "Beach", "Leslie",
+ "Trevithick", "Langston", "Magor", "Shotts", "Howe", "Hunter", "Cross", "Kistler", "Dealtry", "Christner",
+ "Pennington", "Thorley", "Eckhardstein", "Van", "Stroh", "Stough", "Stall", "Beedell", "Shea", "Garland",
+ "Mays", "Pritchard", "Frankenberger", "Rowley", "Lane", "Baum", "Alliman", "Park", "Jardine", "Butler",
+ "Cherry", "Kooser", "Baxter", "Billimek", "Downing", "Hurst", "Wood", "Baird", "Watkins", "Edwards",
+ "Kemerer", "Harding", "Owens", "Eiford", "Keener", "Garneis", "Fiscina", "Mang", "Draudy", "Mills",
+ "Gibson", "Reese", "Todd", "Ramos", "Levett", "Wilks", "Ward", "Mosser", "Dunlap", "Kifer", "Christopher",
+ "Ashbaugh", "Wynter", "Rawls", "Cribbs", "Haynes", "Thigpen", "Schreckengost", "Bishop", "Linton",
+ "Chapman", "James", "Jerome", "Hook", "Omara", "Houston", "Maclagan", "Sandys", "Pickering", "Blois",
+ "Dickson", "Kemble", "Duncan", "Woodward", "Southern", "Henley", "Treeby", "Cram", "Elsas", "Driggers",
+ "Warrick", "Overstreet", "Hindman", "Buck", "Sulyard", "Wentzel", "Swink", "Butt", "Schaeffer",
+ "Hoffhants", "Bould", "Willcox", "Lotherington", "Bagley", "Graff", "White", "Wheeler", "Sloan",
+ "Rodacker", "Hanford", "Jowers", "Kunkle", "Cass", "Powers", "Gilman", "Mcmichaels", "Hobbs", "Herndon",
+ "Prescott", "Smail", "Mcdonald", "Biery", "Orner", "Richards", "Mueller", "Isaman", "Bruxner", "Goodman",
+ "Barth", "Turzanski", "Vorrasi", "Stainforth", "Nehling", "Rahl", "Erschoff", "Greene", "Mckinnon",
+ "Reade", "Smith", "Pery", "Roose", "Greenwood", "Weisgarber", "Curry", "Holts", "Zadovsky", "Parrish",
+ "Putnam", "Munson", "Mcindoe", "Nickolson", "Brooks", "Bollinger", "Stroble", "Siegrist", "Fulton",
+ "Tomey", "Zoucks", "Roberts", "Otis", "Clarke", "Easter", "Johnson", "Fylbrigg", "Taylor", "Swartzbaugh",
+ "Weinstein", "Gadow", "Sayre", "Marcotte", "Wise", "Atweeke", "Mcfall", "Napier", "Eisenhart", "Canham",
+ "Sealis", "Baughman", "Gertraht", "Losey", "Laurence", "Eva", "Pershing", "Kern", "Pirl", "Rega",
+ "Sanborn", "Kanaga", "Sanders", "Anderson", "Dickinson", "Osteen", "Gettemy", "Crom", "Snyder", "Reed",
+ "Laurenzi", "Riggle", "Tillson", "Fowler", "Raub", "Jenner", "Koepple", "Soames", "Goldvogel", "Dimsdale",
+ "Zimmer", "Giesen", "Baker", "Beail", "Mortland", "Bard", "Sanner", "Knopsnider", "Jenkins", "Bailey",
+ "Werner", "Barrett", "Faust", "Agg", "Tomlinson", "Williams", "Little", "Greenawalt", "Wells", "Wilkins",
+ "Gisiko", "Bauerle", "Harrold", "Prechtl", "Polson", "Faast", "Winton", "Garneys", "Peters", "Potter",
+ "Porter", "Tennant", "Eve", "Dugger", "Jones", "Burch", "Cowper", "Whittier" };
+
+ public static String[] firstNames = { "Albert", "Jacquelin", "Dona", "Alia", "Mayme", "Genoveva", "Emma", "Lena",
+ "Melody", "Vilma", "Katelyn", "Jeremy", "Coral", "Leann", "Lita", "Gilda", "Kayla", "Alvina", "Maranda",
+ "Verlie", "Khadijah", "Karey", "Patrice", "Kallie", "Corey", "Mollie", "Daisy", "Melanie", "Sarita",
+ "Nichole", "Pricilla", "Terresa", "Berneice", "Arianne", "Brianne", "Lavinia", "Ulrike", "Lesha", "Adell",
+ "Ardelle", "Marisha", "Laquita", "Karyl", "Maryjane", "Kendall", "Isobel", "Raeann", "Heike", "Barbera",
+ "Norman", "Yasmine", "Nevada", "Mariam", "Edith", "Eugena", "Lovie", "Maren", "Bennie", "Lennie", "Tamera",
+ "Crystal", "Randi", "Anamaria", "Chantal", "Jesenia", "Avis", "Shela", "Randy", "Laurena", "Sharron",
+ "Christiane", "Lorie", "Mario", "Elizabeth", "Reina", "Adria", "Lakisha", "Brittni", "Azzie", "Dori",
+ "Shaneka", "Asuncion", "Katheryn", "Laurice", "Sharita", "Krystal", "Reva", "Inger", "Alpha", "Makeda",
+ "Anabel", "Loni", "Tiara", "Meda", "Latashia", "Leola", "Chin", "Daisey", "Ivory", "Amalia", "Logan",
+ "Tyler", "Kyong", "Carolann", "Maryetta", "Eufemia", "Anya", "Doreatha", "Lorna", "Rutha", "Ehtel",
+ "Debbie", "Chassidy", "Sang", "Christa", "Lottie", "Chun", "Karine", "Peggie", "Amina", "Melany", "Alayna",
+ "Scott", "Romana", "Naomi", "Christiana", "Salena", "Taunya", "Mitsue", "Regina", "Chelsie", "Charity",
+ "Dacia", "Aletha", "Latosha", "Lia", "Tamica", "Chery", "Bianca", "Shu", "Georgianne", "Myriam", "Austin",
+ "Wan", "Mallory", "Jana", "Georgie", "Jenell", "Kori", "Vicki", "Delfina", "June", "Mellisa", "Catherina",
+ "Claudie", "Tynisha", "Dayle", "Enriqueta", "Belen", "Pia", "Sarai", "Rosy", "Renay", "Kacie", "Frieda",
+ "Cayla", "Elissa", "Claribel", "Sabina", "Mackenzie", "Raina", "Cira", "Mitzie", "Aubrey", "Serafina",
+ "Maria", "Katharine", "Esperanza", "Sung", "Daria", "Billye", "Stefanie", "Kasha", "Holly", "Suzanne",
+ "Inga", "Flora", "Andria", "Genevie", "Eladia", "Janet", "Erline", "Renna", "Georgeanna", "Delorse",
+ "Elnora", "Rudy", "Rima", "Leanora", "Letisha", "Love", "Alverta", "Pinkie", "Domonique", "Jeannie",
+ "Jose", "Jacqueline", "Tara", "Lily", "Erna", "Tennille", "Galina", "Tamala", "Kirby", "Nichelle",
+ "Myesha", "Farah", "Santa", "Ludie", "Kenia", "Yee", "Micheline", "Maryann", "Elaina", "Ethelyn",
+ "Emmaline", "Shanell", "Marina", "Nila", "Alane", "Shakira", "Dorris", "Belinda", "Elois", "Barbie",
+ "Carita", "Gisela", "Lura", "Fransisca", "Helga", "Peg", "Leonarda", "Earlie", "Deetta", "Jacquetta",
+ "Blossom", "Kayleigh", "Deloras", "Keshia", "Christinia", "Dulce", "Bernie", "Sheba", "Lashanda", "Tula",
+ "Claretta", "Kary", "Jeanette", "Lupita", "Lenora", "Hisako", "Sherise", "Glynda", "Adela", "Chia",
+ "Sudie", "Mindy", "Caroyln", "Lindsey", "Xiomara", "Mercedes", "Onie", "Loan", "Alexis", "Tommie",
+ "Donette", "Monica", "Soo", "Camellia", "Lavera", "Valery", "Ariana", "Sophia", "Loris", "Ginette",
+ "Marielle", "Tari", "Julissa", "Alesia", "Suzanna", "Emelda", "Erin", "Ladawn", "Sherilyn", "Candice",
+ "Nereida", "Fairy", "Carl", "Joel", "Marilee", "Gracia", "Cordie", "So", "Shanita", "Drew", "Cassie",
+ "Sherie", "Marget", "Norma", "Delois", "Debera", "Chanelle", "Catarina", "Aracely", "Carlene", "Tricia",
+ "Aleen", "Katharina", "Marguerita", "Guadalupe", "Margorie", "Mandie", "Kathe", "Chong", "Sage", "Faith",
+ "Maryrose", "Stephany", "Ivy", "Pauline", "Susie", "Cristen", "Jenifer", "Annette", "Debi", "Karmen",
+ "Luci", "Shayla", "Hope", "Ocie", "Sharie", "Tami", "Breana", "Kerry", "Rubye", "Lashay", "Sondra",
+ "Katrice", "Brunilda", "Cortney", "Yan", "Zenobia", "Penni", "Addie", "Lavona", "Noel", "Anika",
+ "Herlinda", "Valencia", "Bunny", "Tory", "Victoria", "Carrie", "Mikaela", "Wilhelmina", "Chung",
+ "Hortencia", "Gerda", "Wen", "Ilana", "Sibyl", "Candida", "Victorina", "Chantell", "Casie", "Emeline",
+ "Dominica", "Cecila", "Delora", "Miesha", "Nova", "Sally", "Ronald", "Charlette", "Francisca", "Mina",
+ "Jenna", "Loraine", "Felisa", "Lulu", "Page", "Lyda", "Babara", "Flor", "Walter", "Chan", "Sherika",
+ "Kala", "Luna", "Vada", "Syreeta", "Slyvia", "Karin", "Renata", "Robbi", "Glenda", "Delsie", "Lizzie",
+ "Genia", "Caitlin", "Bebe", "Cory", "Sam", "Leslee", "Elva", "Caren", "Kasie", "Leticia", "Shannan",
+ "Vickey", "Sandie", "Kyle", "Chang", "Terrilyn", "Sandra", "Elida", "Marketta", "Elsy", "Tu", "Carman",
+ "Ashlie", "Vernia", "Albertine", "Vivian", "Elba", "Bong", "Margy", "Janetta", "Xiao", "Teofila", "Danyel",
+ "Nickole", "Aleisha", "Tera", "Cleotilde", "Dara", "Paulita", "Isela", "Maricela", "Rozella", "Marivel",
+ "Aurora", "Melissa", "Carylon", "Delinda", "Marvella", "Candelaria", "Deidre", "Tawanna", "Myrtie",
+ "Milagro", "Emilie", "Coretta", "Ivette", "Suzann", "Ammie", "Lucina", "Lory", "Tena", "Eleanor",
+ "Cherlyn", "Tiana", "Brianna", "Myra", "Flo", "Carisa", "Kandi", "Erlinda", "Jacqulyn", "Fermina", "Riva",
+ "Palmira", "Lindsay", "Annmarie", "Tamiko", "Carline", "Amelia", "Quiana", "Lashawna", "Veola", "Belva",
+ "Marsha", "Verlene", "Alex", "Leisha", "Camila", "Mirtha", "Melva", "Lina", "Arla", "Cythia", "Towanda",
+ "Aracelis", "Tasia", "Aurore", "Trinity", "Bernadine", "Farrah", "Deneen", "Ines", "Betty", "Lorretta",
+ "Dorethea", "Hertha", "Rochelle", "Juli", "Shenika", "Yung", "Lavon", "Deeanna", "Nakia", "Lynnette",
+ "Dinorah", "Nery", "Elene", "Carolee", "Mira", "Franchesca", "Lavonda", "Leida", "Paulette", "Dorine",
+ "Allegra", "Keva", "Jeffrey", "Bernardina", "Maryln", "Yoko", "Faviola", "Jayne", "Lucilla", "Charita",
+ "Ewa", "Ella", "Maggie", "Ivey", "Bettie", "Jerri", "Marni", "Bibi", "Sabrina", "Sarah", "Marleen",
+ "Katherin", "Remona", "Jamika", "Antonina", "Oliva", "Lajuana", "Fonda", "Sigrid", "Yael", "Billi",
+ "Verona", "Arminda", "Mirna", "Tesha", "Katheleen", "Bonita", "Kamilah", "Patrica", "Julio", "Shaina",
+ "Mellie", "Denyse", "Deandrea", "Alena", "Meg", "Kizzie", "Krissy", "Karly", "Alleen", "Yahaira", "Lucie",
+ "Karena", "Elaine", "Eloise", "Buena", "Marianela", "Renee", "Nan", "Carolynn", "Windy", "Avril", "Jane",
+ "Vida", "Thea", "Marvel", "Rosaline", "Tifany", "Robena", "Azucena", "Carlota", "Mindi", "Andera", "Jenny",
+ "Courtney", "Lyndsey", "Willette", "Kristie", "Shaniqua", "Tabatha", "Ngoc", "Una", "Marlena", "Louetta",
+ "Vernie", "Brandy", "Jacquelyne", "Jenelle", "Elna", "Erminia", "Ida", "Audie", "Louis", "Marisol",
+ "Shawana", "Harriette", "Karol", "Kitty", "Esmeralda", "Vivienne", "Eloisa", "Iris", "Jeanice", "Cammie",
+ "Jacinda", "Shena", "Floy", "Theda", "Lourdes", "Jayna", "Marg", "Kati", "Tanna", "Rosalyn", "Maxima",
+ "Soon", "Angelika", "Shonna", "Merle", "Kassandra", "Deedee", "Heidi", "Marti", "Renae", "Arleen",
+ "Alfredia", "Jewell", "Carley", "Pennie", "Corina", "Tonisha", "Natividad", "Lilliana", "Darcie", "Shawna",
+ "Angel", "Piedad", "Josefa", "Rebbeca", "Natacha", "Nenita", "Petrina", "Carmon", "Chasidy", "Temika",
+ "Dennise", "Renetta", "Augusta", "Shirlee", "Valeri", "Casimira", "Janay", "Berniece", "Deborah", "Yaeko",
+ "Mimi", "Digna", "Irish", "Cher", "Yong", "Lucila", "Jimmie", "Junko", "Lezlie", "Waneta", "Sandee",
+ "Marquita", "Eura", "Freeda", "Annabell", "Laree", "Jaye", "Wendy", "Toshia", "Kylee", "Aleta", "Emiko",
+ "Clorinda", "Sixta", "Audrea", "Juanita", "Birdie", "Reita", "Latanya", "Nia", "Leora", "Laurine",
+ "Krysten", "Jerrie", "Chantel", "Ira", "Sena", "Andre", "Jann", "Marla", "Precious", "Katy", "Gabrielle",
+ "Yvette", "Brook", "Shirlene", "Eldora", "Laura", "Milda", "Euna", "Jettie", "Debora", "Lise", "Edythe",
+ "Leandra", "Shandi", "Araceli", "Johanne", "Nieves", "Denese", "Carmelita", "Nohemi", "Annice", "Natalie",
+ "Yolande", "Jeffie", "Vashti", "Vickie", "Obdulia", "Youlanda", "Lupe", "Tomoko", "Monserrate", "Domitila",
+ "Etsuko", "Adrienne", "Lakesha", "Melissia", "Odessa", "Meagan", "Veronika", "Jolyn", "Isabelle", "Leah",
+ "Rhiannon", "Gianna", "Audra", "Sommer", "Renate", "Perla", "Thao", "Myong", "Lavette", "Mark", "Emilia",
+ "Ariane", "Karl", "Dorie", "Jacquie", "Mia", "Malka", "Shenita", "Tashina", "Christine", "Cherri", "Roni",
+ "Fran", "Mildred", "Sara", "Clarissa", "Fredia", "Elease", "Samuel", "Earlene", "Vernita", "Mae", "Concha",
+ "Renea", "Tamekia", "Hye", "Ingeborg", "Tessa", "Kelly", "Kristin", "Tam", "Sacha", "Kanisha", "Jillian",
+ "Tiffanie", "Ashlee", "Madelyn", "Donya", "Clementine", "Mickie", "My", "Zena", "Terrie", "Samatha",
+ "Gertie", "Tarra", "Natalia", "Sharlene", "Evie", "Shalon", "Rosalee", "Numbers", "Jodi", "Hattie",
+ "Naoma", "Valene", "Whitley", "Claude", "Alline", "Jeanne", "Camie", "Maragret", "Viola", "Kris", "Marlo",
+ "Arcelia", "Shari", "Jalisa", "Corrie", "Eleonor", "Angelyn", "Merry", "Lauren", "Melita", "Gita",
+ "Elenor", "Aurelia", "Janae", "Lyndia", "Margeret", "Shawanda", "Rolande", "Shirl", "Madeleine", "Celinda",
+ "Jaleesa", "Shemika", "Joye", "Tisa", "Trudie", "Kathrine", "Clarita", "Dinah", "Georgia", "Antoinette",
+ "Janis", "Suzette", "Sherri", "Herta", "Arie", "Hedy", "Cassi", "Audrie", "Caryl", "Jazmine", "Jessica",
+ "Beverly", "Elizbeth", "Marylee", "Londa", "Fredericka", "Argelia", "Nana", "Donnette", "Damaris",
+ "Hailey", "Jamee", "Kathlene", "Glayds", "Lydia", "Apryl", "Verla", "Adam", "Concepcion", "Zelda",
+ "Shonta", "Vernice", "Detra", "Meghann", "Sherley", "Sheri", "Kiyoko", "Margarita", "Adaline", "Mariela",
+ "Velda", "Ailene", "Juliane", "Aiko", "Edyth", "Cecelia", "Shavon", "Florance", "Madeline", "Rheba",
+ "Deann", "Ignacia", "Odelia", "Heide", "Mica", "Jennette", "Maricruz", "Ouida", "Darcy", "Laure",
+ "Justina", "Amada", "Laine", "Cruz", "Sunny", "Francene", "Roxanna", "Nam", "Nancie", "Deanna", "Letty",
+ "Britni", "Kazuko", "Lacresha", "Simon", "Caleb", "Milton", "Colton", "Travis", "Miles", "Jonathan",
+ "Logan", "Rolf", "Emilio", "Roberto", "Marcus", "Tim", "Delmar", "Devon", "Kurt", "Edward", "Jeffrey",
+ "Elvis", "Alfonso", "Blair", "Wm", "Sheldon", "Leonel", "Michal", "Federico", "Jacques", "Leslie",
+ "Augustine", "Hugh", "Brant", "Hong", "Sal", "Modesto", "Curtis", "Jefferey", "Adam", "John", "Glenn",
+ "Vance", "Alejandro", "Refugio", "Lucio", "Demarcus", "Chang", "Huey", "Neville", "Preston", "Bert",
+ "Abram", "Foster", "Jamison", "Kirby", "Erich", "Manual", "Dustin", "Derrick", "Donnie", "Jospeh", "Chris",
+ "Josue", "Stevie", "Russ", "Stanley", "Nicolas", "Samuel", "Waldo", "Jake", "Max", "Ernest", "Reinaldo",
+ "Rene", "Gale", "Morris", "Nathan", "Maximo", "Courtney", "Theodore", "Octavio", "Otha", "Delmer",
+ "Graham", "Dean", "Lowell", "Myles", "Colby", "Boyd", "Adolph", "Jarrod", "Nick", "Mark", "Clinton", "Kim",
+ "Sonny", "Dalton", "Tyler", "Jody", "Orville", "Luther", "Rubin", "Hollis", "Rashad", "Barton", "Vicente",
+ "Ted", "Rick", "Carmine", "Clifton", "Gayle", "Christopher", "Jessie", "Bradley", "Clay", "Theo", "Josh",
+ "Mitchell", "Boyce", "Chung", "Eugenio", "August", "Norbert", "Sammie", "Jerry", "Adan", "Edmundo",
+ "Homer", "Hilton", "Tod", "Kirk", "Emmett", "Milan", "Quincy", "Jewell", "Herb", "Steve", "Carmen",
+ "Bobby", "Odis", "Daron", "Jeremy", "Carl", "Hunter", "Tuan", "Thurman", "Asa", "Brenton", "Shane",
+ "Donny", "Andreas", "Teddy", "Dario", "Cyril", "Hoyt", "Teodoro", "Vincenzo", "Hilario", "Daren",
+ "Agustin", "Marquis", "Ezekiel", "Brendan", "Johnson", "Alden", "Richie", "Granville", "Chad", "Joseph",
+ "Lamont", "Jordon", "Gilberto", "Chong", "Rosendo", "Eddy", "Rob", "Dewitt", "Andre", "Titus", "Russell",
+ "Rigoberto", "Dick", "Garland", "Gabriel", "Hank", "Darius", "Ignacio", "Lazaro", "Johnie", "Mauro",
+ "Edmund", "Trent", "Harris", "Osvaldo", "Marvin", "Judson", "Rodney", "Randall", "Renato", "Richard",
+ "Denny", "Jon", "Doyle", "Cristopher", "Wilson", "Christian", "Jamie", "Roland", "Ken", "Tad", "Romeo",
+ "Seth", "Quinton", "Byron", "Ruben", "Darrel", "Deandre", "Broderick", "Harold", "Ty", "Monroe", "Landon",
+ "Mohammed", "Angel", "Arlen", "Elias", "Andres", "Carlton", "Numbers", "Tony", "Thaddeus", "Issac",
+ "Elmer", "Antoine", "Ned", "Fermin", "Grover", "Benito", "Abdul", "Cortez", "Eric", "Maxwell", "Coy",
+ "Gavin", "Rich", "Andy", "Del", "Giovanni", "Major", "Efren", "Horacio", "Joaquin", "Charles", "Noah",
+ "Deon", "Pasquale", "Reed", "Fausto", "Jermaine", "Irvin", "Ray", "Tobias", "Carter", "Yong", "Jorge",
+ "Brent", "Daniel", "Zane", "Walker", "Thad", "Shaun", "Jaime", "Mckinley", "Bradford", "Nathanial",
+ "Jerald", "Aubrey", "Virgil", "Abel", "Philip", "Chester", "Chadwick", "Dominick", "Britt", "Emmitt",
+ "Ferdinand", "Julian", "Reid", "Santos", "Dwain", "Morgan", "James", "Marion", "Micheal", "Eddie", "Brett",
+ "Stacy", "Kerry", "Dale", "Nicholas", "Darrick", "Freeman", "Scott", "Newton", "Sherman", "Felton",
+ "Cedrick", "Winfred", "Brad", "Fredric", "Dewayne", "Virgilio", "Reggie", "Edgar", "Heriberto", "Shad",
+ "Timmy", "Javier", "Nestor", "Royal", "Lynn", "Irwin", "Ismael", "Jonas", "Wiley", "Austin", "Kieth",
+ "Gonzalo", "Paris", "Earnest", "Arron", "Jarred", "Todd", "Erik", "Maria", "Chauncey", "Neil", "Conrad",
+ "Maurice", "Roosevelt", "Jacob", "Sydney", "Lee", "Basil", "Louis", "Rodolfo", "Rodger", "Roman", "Corey",
+ "Ambrose", "Cristobal", "Sylvester", "Benton", "Franklin", "Marcelo", "Guillermo", "Toby", "Jeramy",
+ "Donn", "Danny", "Dwight", "Clifford", "Valentine", "Matt", "Jules", "Kareem", "Ronny", "Lonny", "Son",
+ "Leopoldo", "Dannie", "Gregg", "Dillon", "Orlando", "Weston", "Kermit", "Damian", "Abraham", "Walton",
+ "Adrian", "Rudolf", "Will", "Les", "Norberto", "Fred", "Tyrone", "Ariel", "Terry", "Emmanuel", "Anderson",
+ "Elton", "Otis", "Derek", "Frankie", "Gino", "Lavern", "Jarod", "Kenny", "Dane", "Keenan", "Bryant",
+ "Eusebio", "Dorian", "Ali", "Lucas", "Wilford", "Jeremiah", "Warner", "Woodrow", "Galen", "Bob",
+ "Johnathon", "Amado", "Michel", "Harry", "Zachery", "Taylor", "Booker", "Hershel", "Mohammad", "Darrell",
+ "Kyle", "Stuart", "Marlin", "Hyman", "Jeffery", "Sidney", "Merrill", "Roy", "Garrett", "Porter", "Kenton",
+ "Giuseppe", "Terrance", "Trey", "Felix", "Buster", "Von", "Jackie", "Linwood", "Darron", "Francisco",
+ "Bernie", "Diego", "Brendon", "Cody", "Marco", "Ahmed", "Antonio", "Vince", "Brooks", "Kendrick", "Ross",
+ "Mohamed", "Jim", "Benny", "Gerald", "Pablo", "Charlie", "Antony", "Werner", "Hipolito", "Minh", "Mel",
+ "Derick", "Armand", "Fidel", "Lewis", "Donnell", "Desmond", "Vaughn", "Guadalupe", "Keneth", "Rodrick",
+ "Spencer", "Chas", "Gus", "Harlan", "Wes", "Carmelo", "Jefferson", "Gerard", "Jarvis", "Haywood", "Hayden",
+ "Sergio", "Gene", "Edgardo", "Colin", "Horace", "Dominic", "Aldo", "Adolfo", "Juan", "Man", "Lenard",
+ "Clement", "Everett", "Hal", "Bryon", "Mason", "Emerson", "Earle", "Laurence", "Columbus", "Lamar",
+ "Douglas", "Ian", "Fredrick", "Marc", "Loren", "Wallace", "Randell", "Noble", "Ricardo", "Rory", "Lindsey",
+ "Boris", "Bill", "Carlos", "Domingo", "Grant", "Craig", "Ezra", "Matthew", "Van", "Rudy", "Danial",
+ "Brock", "Maynard", "Vincent", "Cole", "Damion", "Ellsworth", "Marcel", "Markus", "Rueben", "Tanner",
+ "Reyes", "Hung", "Kennith", "Lindsay", "Howard", "Ralph", "Jed", "Monte", "Garfield", "Avery", "Bernardo",
+ "Malcolm", "Sterling", "Ezequiel", "Kristofer", "Luciano", "Casey", "Rosario", "Ellis", "Quintin",
+ "Trevor", "Miquel", "Jordan", "Arthur", "Carson", "Tyron", "Grady", "Walter", "Jonathon", "Ricky",
+ "Bennie", "Terrence", "Dion", "Dusty", "Roderick", "Isaac", "Rodrigo", "Harrison", "Zack", "Dee", "Devin",
+ "Rey", "Ulysses", "Clint", "Greg", "Dino", "Frances", "Wade", "Franklyn", "Jude", "Bradly", "Salvador",
+ "Rocky", "Weldon", "Lloyd", "Milford", "Clarence", "Alec", "Allan", "Bobbie", "Oswaldo", "Wilfred",
+ "Raleigh", "Shelby", "Willy", "Alphonso", "Arnoldo", "Robbie", "Truman", "Nicky", "Quinn", "Damien",
+ "Lacy", "Marcos", "Parker", "Burt", "Carroll", "Denver", "Buck", "Dong", "Normand", "Billie", "Edwin",
+ "Troy", "Arden", "Rusty", "Tommy", "Kenneth", "Leo", "Claud", "Joel", "Kendall", "Dante", "Milo", "Cruz",
+ "Lucien", "Ramon", "Jarrett", "Scottie", "Deshawn", "Ronnie", "Pete", "Alonzo", "Whitney", "Stefan",
+ "Sebastian", "Edmond", "Enrique", "Branden", "Leonard", "Loyd", "Olin", "Ron", "Rhett", "Frederic",
+ "Orval", "Tyrell", "Gail", "Eli", "Antonia", "Malcom", "Sandy", "Stacey", "Nickolas", "Hosea", "Santo",
+ "Oscar", "Fletcher", "Dave", "Patrick", "Dewey", "Bo", "Vito", "Blaine", "Randy", "Robin", "Winston",
+ "Sammy", "Edwardo", "Manuel", "Valentin", "Stanford", "Filiberto", "Buddy", "Zachariah", "Johnnie",
+ "Elbert", "Paul", "Isreal", "Jerrold", "Leif", "Owen", "Sung", "Junior", "Raphael", "Josef", "Donte",
+ "Allen", "Florencio", "Raymond", "Lauren", "Collin", "Eliseo", "Bruno", "Martin", "Lyndon", "Kurtis",
+ "Salvatore", "Erwin", "Michael", "Sean", "Davis", "Alberto", "King", "Rolland", "Joe", "Tory", "Chase",
+ "Dallas", "Vernon", "Beau", "Terrell", "Reynaldo", "Monty", "Jame", "Dirk", "Florentino", "Reuben", "Saul",
+ "Emory", "Esteban", "Michale", "Claudio", "Jacinto", "Kelley", "Levi", "Andrea", "Lanny", "Wendell",
+ "Elwood", "Joan", "Felipe", "Palmer", "Elmo", "Lawrence", "Hubert", "Rudolph", "Duane", "Cordell",
+ "Everette", "Mack", "Alan", "Efrain", "Trenton", "Bryan", "Tom", "Wilmer", "Clyde", "Chance", "Lou",
+ "Brain", "Justin", "Phil", "Jerrod", "George", "Kris", "Cyrus", "Emery", "Rickey", "Lincoln", "Renaldo",
+ "Mathew", "Luke", "Dwayne", "Alexis", "Jackson", "Gil", "Marty", "Burton", "Emil", "Glen", "Willian",
+ "Clemente", "Keven", "Barney", "Odell", "Reginald", "Aurelio", "Damon", "Ward", "Gustavo", "Harley",
+ "Peter", "Anibal", "Arlie", "Nigel", "Oren", "Zachary", "Scot", "Bud", "Wilbert", "Bart", "Josiah",
+ "Marlon", "Eldon", "Darryl", "Roger", "Anthony", "Omer", "Francis", "Patricia", "Moises", "Chuck",
+ "Waylon", "Hector", "Jamaal", "Cesar", "Julius", "Rex", "Norris", "Ollie", "Isaias", "Quentin", "Graig",
+ "Lyle", "Jeffry", "Karl", "Lester", "Danilo", "Mike", "Dylan", "Carlo", "Ryan", "Leon", "Percy", "Lucius",
+ "Jamel", "Lesley", "Joey", "Cornelius", "Rico", "Arnulfo", "Chet", "Margarito", "Ernie", "Nathanael",
+ "Amos", "Cleveland", "Luigi", "Alfonzo", "Phillip", "Clair", "Elroy", "Alva", "Hans", "Shon", "Gary",
+ "Jesus", "Cary", "Silas", "Keith", "Israel", "Willard", "Randolph", "Dan", "Adalberto", "Claude",
+ "Delbert", "Garry", "Mary", "Larry", "Riley", "Robt", "Darwin", "Barrett", "Steven", "Kelly", "Herschel",
+ "Darnell", "Scotty", "Armando", "Miguel", "Lawerence", "Wesley", "Garth", "Carol", "Micah", "Alvin",
+ "Billy", "Earl", "Pat", "Brady", "Cory", "Carey", "Bernard", "Jayson", "Nathaniel", "Gaylord", "Archie",
+ "Dorsey", "Erasmo", "Angelo", "Elisha", "Long", "Augustus", "Hobert", "Drew", "Stan", "Sherwood",
+ "Lorenzo", "Forrest", "Shawn", "Leigh", "Hiram", "Leonardo", "Gerry", "Myron", "Hugo", "Alvaro", "Leland",
+ "Genaro", "Jamey", "Stewart", "Elden", "Irving", "Olen", "Antone", "Freddy", "Lupe", "Joshua", "Gregory",
+ "Andrew", "Sang", "Wilbur", "Gerardo", "Merlin", "Williams", "Johnny", "Alex", "Tommie", "Jimmy",
+ "Donovan", "Dexter", "Gaston", "Tracy", "Jeff", "Stephen", "Berry", "Anton", "Darell", "Fritz", "Willis",
+ "Noel", "Mariano", "Crawford", "Zoey", "Alex", "Brianna", "Carlie", "Lloyd", "Cal", "Astor", "Randolf",
+ "Magdalene", "Trevelyan", "Terance", "Roy", "Kermit", "Harriett", "Crystal", "Laurinda", "Kiersten",
+ "Phyllida", "Liz", "Bettie", "Rena", "Colten", "Berenice", "Sindy", "Wilma", "Amos", "Candi", "Ritchie",
+ "Dirk", "Kathlyn", "Callista", "Anona", "Flossie", "Sterling", "Calista", "Regan", "Erica", "Jeana",
+ "Keaton", "York", "Nolan", "Daniel", "Benton", "Tommie", "Serenity", "Deanna", "Chas", "Heron", "Marlyn",
+ "Xylia", "Tristin", "Lyndon", "Andriana", "Madelaine", "Maddison", "Leila", "Chantelle", "Audrey",
+ "Connor", "Daley", "Tracee", "Tilda", "Eliot", "Merle", "Linwood", "Kathryn", "Silas", "Alvina",
+ "Phinehas", "Janis", "Alvena", "Zubin", "Gwendolen", "Caitlyn", "Bertram", "Hailee", "Idelle", "Homer",
+ "Jannah", "Delbert", "Rhianna", "Cy", "Jefferson", "Wayland", "Nona", "Tempest", "Reed", "Jenifer",
+ "Ellery", "Nicolina", "Aldous", "Prince", "Lexia", "Vinnie", "Doug", "Alberic", "Kayleen", "Woody",
+ "Rosanne", "Ysabel", "Skyler", "Twyla", "Geordie", "Leta", "Clive", "Aaron", "Scottie", "Celeste", "Chuck",
+ "Erle", "Lallie", "Jaycob", "Ray", "Carrie", "Laurita", "Noreen", "Meaghan", "Ulysses", "Andy", "Drogo",
+ "Dina", "Yasmin", "Mya", "Luvenia", "Urban", "Jacob", "Laetitia", "Sherry", "Love", "Michaela", "Deonne",
+ "Summer", "Brendon", "Sheena", "Mason", "Jayson", "Linden", "Salal", "Darrell", "Diana", "Hudson",
+ "Lennon", "Isador", "Charley", "April", "Ralph", "James", "Mina", "Jolyon", "Laurine", "Monna", "Carita",
+ "Munro", "Elsdon", "Everette", "Radclyffe", "Darrin", "Herbert", "Gawain", "Sheree", "Trudy", "Emmaline",
+ "Kassandra", "Rebecca", "Basil", "Jen", "Don", "Osborne", "Lilith", "Hannah", "Fox", "Rupert", "Paulene",
+ "Darius", "Wally", "Baptist", "Sapphire", "Tia", "Sondra", "Kylee", "Ashton", "Jepson", "Joetta", "Val",
+ "Adela", "Zacharias", "Zola", "Marmaduke", "Shannah", "Posie", "Oralie", "Brittany", "Ernesta", "Raymund",
+ "Denzil", "Daren", "Roosevelt", "Nelson", "Fortune", "Mariel", "Nick", "Jaden", "Upton", "Oz", "Margaux",
+ "Precious", "Albert", "Bridger", "Jimmy", "Nicola", "Rosalynne", "Keith", "Walt", "Della", "Joanna",
+ "Xenia", "Esmeralda", "Major", "Simon", "Rexana", "Stacy", "Calanthe", "Sherley", "Kaitlyn", "Graham",
+ "Ramsey", "Abbey", "Madlyn", "Kelvin", "Bill", "Rue", "Monica", "Caileigh", "Laraine", "Booker", "Jayna",
+ "Greta", "Jervis", "Sherman", "Kendrick", "Tommy", "Iris", "Geffrey", "Kaelea", "Kerr", "Garrick", "Jep",
+ "Audley", "Nic", "Bronte", "Beulah", "Patricia", "Jewell", "Deidra", "Cory", "Everett", "Harper",
+ "Charity", "Godfrey", "Jaime", "Sinclair", "Talbot", "Dayna", "Cooper", "Rosaline", "Jennie", "Eileen",
+ "Latanya", "Corinna", "Roxie", "Caesar", "Charles", "Pollie", "Lindsey", "Sorrel", "Dwight", "Jocelyn",
+ "Weston", "Shyla", "Valorie", "Bessie", "Josh", "Lessie", "Dayton", "Kathi", "Chasity", "Wilton", "Adam",
+ "William", "Ash", "Angela", "Ivor", "Ria", "Jazmine", "Hailey", "Jo", "Silvestra", "Ernie", "Clifford",
+ "Levi", "Matilda", "Quincey", "Camilla", "Delicia", "Phemie", "Laurena", "Bambi", "Lourdes", "Royston",
+ "Chastity", "Lynwood", "Elle", "Brenda", "Phoebe", "Timothy", "Raschelle", "Lilly", "Burt", "Rina",
+ "Rodney", "Maris", "Jaron", "Wilf", "Harlan", "Audra", "Vincent", "Elwyn", "Drew", "Wynter", "Ora",
+ "Lissa", "Virgil", "Xavier", "Chad", "Ollie", "Leyton", "Karolyn", "Skye", "Roni", "Gladys", "Dinah",
+ "Penny", "August", "Osmund", "Whitaker", "Brande", "Cornell", "Phil", "Zara", "Kilie", "Gavin", "Coty",
+ "Randy", "Teri", "Keira", "Pru", "Clemency", "Kelcey", "Nevil", "Poppy", "Gareth", "Christabel", "Bastian",
+ "Wynonna", "Roselyn", "Goddard", "Collin", "Trace", "Neal", "Effie", "Denys", "Virginia", "Richard",
+ "Isiah", "Harrietta", "Gaylord", "Diamond", "Trudi", "Elaine", "Jemmy", "Gage", "Annabel", "Quincy", "Syd",
+ "Marianna", "Philomena", "Aubree", "Kathie", "Jacki", "Kelley", "Bess", "Cecil", "Maryvonne", "Kassidy",
+ "Anselm", "Dona", "Darby", "Jamison", "Daryl", "Darell", "Teal", "Lennie", "Bartholomew", "Katie",
+ "Maybelline", "Kimball", "Elvis", "Les", "Flick", "Harley", "Beth", "Bidelia", "Montague", "Helen", "Ozzy",
+ "Stef", "Debra", "Maxene", "Stefanie", "Russ", "Avril", "Johnathan", "Orson", "Chelsey", "Josephine",
+ "Deshaun", "Wendell", "Lula", "Ferdinanda", "Greg", "Brad", "Kynaston", "Dena", "Russel", "Robertina",
+ "Misti", "Leon", "Anjelica", "Bryana", "Myles", "Judi", "Curtis", "Davin", "Kristia", "Chrysanta",
+ "Hayleigh", "Hector", "Osbert", "Eustace", "Cary", "Tansy", "Cayley", "Maryann", "Alissa", "Ike",
+ "Tranter", "Reina", "Alwilda", "Sidony", "Columbine", "Astra", "Jillie", "Stephania", "Jonah", "Kennedy",
+ "Ferdinand", "Allegria", "Donella", "Kelleigh", "Darian", "Eldreda", "Jayden", "Herbie", "Jake", "Winston",
+ "Vi", "Annie", "Cherice", "Hugo", "Tricia", "Haydee", "Cassarah", "Darden", "Mallory", "Alton", "Hadley",
+ "Romayne", "Lacey", "Ern", "Alayna", "Cecilia", "Seward", "Tilly", "Edgar", "Concordia", "Ibbie", "Dahlia",
+ "Oswin", "Stu", "Brett", "Maralyn", "Kristeen", "Dotty", "Robyn", "Nessa", "Tresha", "Guinevere",
+ "Emerson", "Haze", "Lyn", "Henderson", "Lexa", "Jaylen", "Gail", "Lizette", "Tiara", "Robbie", "Destiny",
+ "Alice", "Livia", "Rosy", "Leah", "Jan", "Zach", "Vita", "Gia", "Micheal", "Rowina", "Alysha", "Bobbi",
+ "Delores", "Osmond", "Karaugh", "Wilbur", "Kasandra", "Renae", "Kaety", "Dora", "Gaye", "Amaryllis",
+ "Katelyn", "Dacre", "Prudence", "Ebony", "Camron", "Jerrold", "Vivyan", "Randall", "Donna", "Misty",
+ "Damon", "Selby", "Esmund", "Rian", "Garry", "Julius", "Raelene", "Clement", "Dom", "Tibby", "Moss",
+ "Millicent", "Gwendoline", "Berry", "Ashleigh", "Lilac", "Quin", "Vere", "Creighton", "Harriet", "Malvina",
+ "Lianne", "Pearle", "Kizzie", "Kara", "Petula", "Jeanie", "Maria", "Pacey", "Victoria", "Huey", "Toni",
+ "Rose", "Wallis", "Diggory", "Josiah", "Delma", "Keysha", "Channing", "Prue", "Lee", "Ryan", "Sidney",
+ "Valerie", "Clancy", "Ezra", "Gilbert", "Clare", "Laz", "Crofton", "Mike", "Annabella", "Tara", "Eldred",
+ "Arthur", "Jaylon", "Peronel", "Paden", "Dot", "Marian", "Amyas", "Alexus", "Esmond", "Abbie", "Stanley",
+ "Brittani", "Vickie", "Errol", "Kimberlee", "Uland", "Ebenezer", "Howie", "Eveline", "Andrea", "Trish",
+ "Hopkin", "Bryanna", "Temperance", "Valarie", "Femie", "Alix", "Terrell", "Lewin", "Lorrin", "Happy",
+ "Micah", "Rachyl", "Sloan", "Gertrude", "Elizabeth", "Dorris", "Andra", "Bram", "Gary", "Jeannine",
+ "Maurene", "Irene", "Yolonda", "Jonty", "Coleen", "Cecelia", "Chantal", "Stuart", "Caris", "Ros",
+ "Kaleigh", "Mirabelle", "Kolby", "Primrose", "Susannah", "Ginny", "Jinny", "Dolly", "Lettice", "Sonny",
+ "Melva", "Ernest", "Garret", "Reagan", "Trenton", "Gallagher", "Edwin", "Nikolas", "Corrie", "Lynette",
+ "Ettie", "Sly", "Debbi", "Eudora", "Brittney", "Tacey", "Marius", "Anima", "Gordon", "Olivia", "Kortney",
+ "Shantel", "Kolleen", "Nevaeh", "Buck", "Sera", "Liliana", "Aric", "Kalyn", "Mick", "Libby", "Ingram",
+ "Alexandria", "Darleen", "Jacklyn", "Hughie", "Tyler", "Aida", "Ronda", "Deemer", "Taryn", "Laureen",
+ "Samantha", "Dave", "Hardy", "Baldric", "Montgomery", "Gus", "Ellis", "Titania", "Luke", "Chase", "Haidee",
+ "Mayra", "Isabell", "Trinity", "Milo", "Abigail", "Tacita", "Meg", "Hervey", "Natasha", "Sadie", "Holden",
+ "Dee", "Mansel", "Perry", "Randi", "Frederica", "Georgina", "Kolour", "Debbie", "Seraphina", "Elspet",
+ "Julyan", "Raven", "Zavia", "Jarvis", "Jaymes", "Grover", "Cairo", "Alea", "Jordon", "Braxton", "Donny",
+ "Rhoda", "Tonya", "Bee", "Alyssia", "Ashlyn", "Reanna", "Lonny", "Arlene", "Deb", "Jane", "Nikole",
+ "Bettina", "Harrison", "Tamzen", "Arielle", "Adelaide", "Faith", "Bridie", "Wilburn", "Fern", "Nan",
+ "Shaw", "Zeke", "Alan", "Dene", "Gina", "Alexa", "Bailey", "Sal", "Tammy", "Maximillian", "America",
+ "Sylvana", "Fitz", "Mo", "Marissa", "Cass", "Eldon", "Wilfrid", "Tel", "Joann", "Kendra", "Tolly",
+ "Leanne", "Ferdie", "Haven", "Lucas", "Marlee", "Cyrilla", "Red", "Phoenix", "Jazmin", "Carin", "Gena",
+ "Lashonda", "Tucker", "Genette", "Kizzy", "Winifred", "Melody", "Keely", "Kaylyn", "Radcliff", "Lettie",
+ "Foster", "Lyndsey", "Nicholas", "Farley", "Louisa", "Dana", "Dortha", "Francine", "Doran", "Bonita",
+ "Hal", "Sawyer", "Reginald", "Aislin", "Nathan", "Baylee", "Abilene", "Ladonna", "Maurine", "Shelly",
+ "Deandre", "Jasmin", "Roderic", "Tiffany", "Amanda", "Verity", "Wilford", "Gayelord", "Whitney", "Demelza",
+ "Kenton", "Alberta", "Kyra", "Tabitha", "Sampson", "Korey", "Lillian", "Edison", "Clayton", "Steph",
+ "Maya", "Dusty", "Jim", "Ronny", "Adrianne", "Bernard", "Harris", "Kiley", "Alexander", "Kisha", "Ethalyn",
+ "Patience", "Briony", "Indigo", "Aureole", "Makenzie", "Molly", "Sherilyn", "Barry", "Laverne", "Hunter",
+ "Rocky", "Tyreek", "Madalyn", "Phyliss", "Chet", "Beatrice", "Faye", "Lavina", "Madelyn", "Tracey",
+ "Gyles", "Patti", "Carlyn", "Stephanie", "Jackalyn", "Larrie", "Kimmy", "Isolda", "Emelina", "Lis",
+ "Zillah", "Cody", "Sheard", "Rufus", "Paget", "Mae", "Rexanne", "Luvinia", "Tamsen", "Rosanna", "Greig",
+ "Stacia", "Mabelle", "Quianna", "Lotus", "Delice", "Bradford", "Angus", "Cosmo", "Earlene", "Adrian",
+ "Arlie", "Noelle", "Sabella", "Isa", "Adelle", "Innocent", "Kirby", "Trixie", "Kenelm", "Nelda", "Melia",
+ "Kendal", "Dorinda", "Placid", "Linette", "Kam", "Sherisse", "Evan", "Ewart", "Janice", "Linton",
+ "Jacaline", "Charissa", "Douglas", "Aileen", "Kemp", "Oli", "Amethyst", "Rosie", "Nigella", "Sherill",
+ "Anderson", "Alanna", "Eric", "Claudia", "Jennifer", "Boniface", "Harriet", "Vernon", "Lucy", "Shawnee",
+ "Gerard", "Cecily", "Romey", "Randall", "Wade", "Lux", "Dawson", "Gregg", "Kade", "Roxanne", "Melinda",
+ "Rolland", "Rowanne", "Fannie", "Isidore", "Melia", "Harvie", "Salal", "Eleonor", "Jacquette", "Lavone",
+ "Shanika", "Tarquin", "Janet", "Josslyn", "Maegan", "Augusta", "Aubree", "Francene", "Martie", "Marisa",
+ "Tyreek", "Tatianna", "Caleb", "Sheridan", "Nellie", "Barbara", "Wat", "Jayla", "Esmaralda", "Graeme",
+ "Lavena", "Jemima", "Nikolas", "Triston", "Portia", "Kyla", "Marcus", "Raeburn", "Jamison", "Earl", "Wren",
+ "Leighton", "Lagina", "Lucasta", "Dina", "Amaranta", "Jessika", "Claud", "Bernard", "Winifred", "Ebba",
+ "Sammi", "Gall", "Chloe", "Ottoline", "Herbert", "Janice", "Gareth", "Channing", "Caleigh", "Kailee",
+ "Ralphie", "Tamzen", "Quincy", "Beaumont", "Albert", "Jadyn", "Violet", "Luanna", "Moriah", "Humbert",
+ "Jed", "Leona", "Hale", "Mitch", "Marlin", "Nivek", "Darwin", "Dirk", "Liliana", "Meadow", "Bernadine",
+ "Jorie", "Peyton", "Astra", "Roscoe", "Gina", "Lovell", "Jewel", "Romayne", "Rosy", "Imogene",
+ "Margaretta", "Lorinda", "Hopkin", "Bobby", "Flossie", "Bennie", "Horatio", "Jonah", "Lyn", "Deana",
+ "Juliana", "Blanch", "Wright", "Kendal", "Woodrow", "Tania", "Austyn", "Val", "Mona", "Charla", "Rudyard",
+ "Pamela", "Raven", "Zena", "Nicola", "Kaelea", "Conor", "Virgil", "Sonnie", "Goodwin", "Christianne",
+ "Linford", "Myron", "Denton", "Charita", "Brody", "Ginnie", "Harrison", "Jeanine", "Quin", "Isolda",
+ "Zoie", "Pearce", "Margie", "Larrie", "Angelina", "Marcia", "Jessamine", "Delilah", "Dick", "Luana",
+ "Delicia", "Lake", "Luvenia", "Vaughan", "Concordia", "Gayelord", "Cheyenne", "Felix", "Dorris", "Pen",
+ "Kristeen", "Parris", "Everitt", "Josephina", "Amy", "Tommie", "Adrian", "April", "Rosaline", "Zachery",
+ "Trace", "Phoebe", "Jenelle", "Kameron", "Katharine", "Media", "Colton", "Tad", "Quianna", "Kerenza",
+ "Greta", "Luvinia", "Pete", "Tonya", "Beckah", "Barbra", "Jon", "Tetty", "Corey", "Sylvana", "Kizzy",
+ "Korey", "Trey", "Haydee", "Penny", "Mandy", "Panda", "Coline", "Ramsey", "Sukie", "Annabel", "Sarina",
+ "Corbin", "Suzanna", "Rob", "Duana", "Shell", "Jason", "Eddy", "Rube", "Roseann", "Celia", "Brianne",
+ "Nerissa", "Jera", "Humphry", "Ashlynn", "Terrence", "Philippina", "Coreen", "Kolour", "Indiana", "Paget",
+ "Marlyn", "Hester", "Isbel", "Ocean", "Harris", "Leslie", "Vere", "Monroe", "Isabelle", "Bertie", "Clitus",
+ "Dave", "Alethea", "Lessie", "Louiza", "Madlyn", "Garland", "Wolf", "Lalo", "Donny", "Amabel", "Tianna",
+ "Louie", "Susie", "Mackenzie", "Renie", "Tess", "Marmaduke", "Gwendolen", "Bettina", "Beatrix", "Esmund",
+ "Minnie", "Carlie", "Barnabas", "Ruthie", "Honour", "Haylie", "Xavior", "Freddie", "Ericka", "Aretha",
+ "Edie", "Madelina", "Anson", "Tabby", "Derrick", "Jocosa", "Deirdre", "Aislin", "Chastity", "Abigail",
+ "Wynonna", "Zo", "Eldon", "Krystine", "Ghislaine", "Zavia", "Nolene", "Marigold", "Kelley", "Sylvester",
+ "Odell", "George", "Laurene", "Franklyn", "Clarice", "Mo", "Dustin", "Debbi", "Lina", "Tony", "Acacia",
+ "Hettie", "Natalee", "Marcie", "Brittany", "Elnora", "Rachel", "Dawn", "Basil", "Christal", "Anjelica",
+ "Fran", "Tawny", "Delroy", "Tameka", "Lillie", "Ceara", "Deanna", "Deshaun", "Ken", "Bradford", "Justina",
+ "Merle", "Draven", "Gretta", "Harriette", "Webster", "Nathaniel", "Anemone", "Coleen", "Ruth", "Chryssa",
+ "Hortensia", "Saffie", "Deonne", "Leopold", "Harlan", "Lea", "Eppie", "Lucinda", "Tilda", "Fanny", "Titty",
+ "Lockie", "Jepson", "Sherisse", "Maralyn", "Ethel", "Sly", "Ebenezer", "Canute", "Ella", "Freeman",
+ "Reuben", "Olivette", "Nona", "Rik", "Amice", "Kristine", "Kathie", "Jayne", "Jeri", "Mckenna", "Bertram",
+ "Kaylee", "Livia", "Gil", "Wallace", "Maryann", "Keeleigh", "Laurinda", "Doran", "Khloe", "Dakota",
+ "Yaron", "Kimberleigh", "Gytha", "Doris", "Marylyn", "Benton", "Linnette", "Esther", "Jakki", "Rowina",
+ "Marian", "Roselyn", "Norbert", "Maggie", "Caesar", "Phinehas", "Jerry", "Jasmine", "Antonette", "Miriam",
+ "Monna", "Maryvonne", "Jacquetta", "Bernetta", "Napier", "Annie", "Gladwin", "Sheldon", "Aric", "Elouise",
+ "Gawain", "Kristia", "Gabe", "Kyra", "Red", "Tod", "Dudley", "Lorraine", "Ryley", "Sabina", "Poppy",
+ "Leland", "Aileen", "Eglantine", "Alicia", "Jeni", "Addy", "Tiffany", "Geffrey", "Lavina", "Collin",
+ "Clover", "Vin", "Jerome", "Doug", "Vincent", "Florence", "Scarlet", "Celeste", "Desdemona", "Tiphanie",
+ "Kassandra", "Ashton", "Madison", "Art", "Magdalene", "Iona", "Josepha", "Anise", "Ferne", "Derek",
+ "Huffie", "Qiana", "Ysabel", "Tami", "Shannah", "Xavier", "Willard", "Winthrop", "Vickie", "Maura",
+ "Placid", "Tiara", "Reggie", "Elissa", "Isa", "Chrysanta", "Jeff", "Bessie", "Terri", "Amilia", "Brett",
+ "Daniella", "Damion", "Carolina", "Maximillian", "Travers", "Benjamin", "Oprah", "Darcy", "Yolanda",
+ "Nicolina", "Crofton", "Jarrett", "Kaitlin", "Shauna", "Keren", "Bevis", "Kalysta", "Sharron", "Alyssa",
+ "Blythe", "Zelma", "Caelie", "Norwood", "Billie", "Patrick", "Gary", "Cambria", "Tylar", "Mason", "Helen",
+ "Melyssa", "Gene", "Gilberta", "Carter", "Herbie", "Harmonie", "Leola", "Eugenia", "Clint", "Pauletta",
+ "Edwyna", "Georgina", "Teal", "Harper", "Izzy", "Dillon", "Kezia", "Evangeline", "Colene", "Madelaine",
+ "Zilla", "Rudy", "Dottie", "Caris", "Morton", "Marge", "Tacey", "Parker", "Troy", "Liza", "Lewin",
+ "Tracie", "Justine", "Dallas", "Linden", "Ray", "Loretta", "Teri", "Elvis", "Diane", "Julianna", "Manfred",
+ "Denise", "Eireen", "Ann", "Kenith", "Linwood", "Kathlyn", "Bernice", "Shelley", "Oswald", "Amedeus",
+ "Homer", "Tanzi", "Ted", "Ralphina", "Hyacinth", "Lotus", "Matthias", "Arlette", "Clark", "Cecil",
+ "Elspeth", "Alvena", "Noah", "Millard", "Brenden", "Cole", "Philipa", "Nina", "Thelma", "Iantha", "Reid",
+ "Jefferson", "Meg", "Elsie", "Shirlee", "Nathan", "Nancy", "Simona", "Racheal", "Carin", "Emory", "Delice",
+ "Kristi", "Karaugh", "Kaety", "Tilly", "Em", "Alanis", "Darrin", "Jerrie", "Hollis", "Cary", "Marly",
+ "Carita", "Jody", "Farley", "Hervey", "Rosalin", "Cuthbert", "Stewart", "Jodene", "Caileigh", "Briscoe",
+ "Dolores", "Sheree", "Eustace", "Nigel", "Detta", "Barret", "Rowland", "Kenny", "Githa", "Zoey", "Adela",
+ "Petronella", "Opal", "Coleman", "Niles", "Cyril", "Dona", "Alberic", "Allannah", "Jules", "Avalon",
+ "Hadley", "Thomas", "Renita", "Calanthe", "Heron", "Shawnda", "Chet", "Malina", "Manny", "Rina", "Frieda",
+ "Eveleen", "Deshawn", "Amos", "Raelene", "Paige", "Molly", "Nannie", "Ileen", "Brendon", "Milford",
+ "Unice", "Rebeccah", "Caedmon", "Gae", "Doreen", "Vivian", "Louis", "Raphael", "Vergil", "Lise", "Glenn",
+ "Karyn", "Terance", "Reina", "Jake", "Gordon", "Wisdom", "Isiah", "Gervase", "Fern", "Marylou", "Roddy",
+ "Justy", "Derick", "Shantelle", "Adam", "Chantel", "Madoline", "Emmerson", "Lexie", "Mickey", "Stephen",
+ "Dane", "Stacee", "Elwin", "Tracey", "Alexandra", "Ricky", "Ian", "Kasey", "Rita", "Alanna", "Georgene",
+ "Deon", "Zavier", "Ophelia", "Deforest", "Lowell", "Zubin", "Hardy", "Osmund", "Tabatha", "Debby",
+ "Katlyn", "Tallulah", "Priscilla", "Braden", "Wil", "Keziah", "Jen", "Aggie", "Korbin", "Lemoine",
+ "Barnaby", "Tranter", "Goldie", "Roderick", "Trina", "Emery", "Pris", "Sidony", "Adelle", "Tate", "Wilf",
+ "Zola", "Brande", "Chris", "Calanthia", "Lilly", "Kaycee", "Lashonda", "Jasmin", "Elijah", "Shantel",
+ "Simon", "Rosalind", "Jarod", "Kaylie", "Corrine", "Joselyn", "Archibald", "Mariabella", "Winton",
+ "Merlin", "Chad", "Ursula", "Kristopher", "Hewie", "Adrianna", "Lyndsay", "Jasmyn", "Tim", "Evette",
+ "Margaret", "Samson", "Bronte", "Terence", "Leila", "Candice", "Tori", "Jamey", "Coriander", "Conrad",
+ "Floyd", "Karen", "Lorin", "Maximilian", "Cairo", "Emily", "Yasmin", "Karolyn", "Bryan", "Lanny",
+ "Kimberly", "Rick", "Chaz", "Krystle", "Lyric", "Laura", "Garrick", "Flip", "Monty", "Brendan",
+ "Ermintrude", "Rayner", "Merla", "Titus", "Marva", "Patricia", "Leone", "Tracy", "Jaqueline", "Hallam",
+ "Delores", "Cressida", "Carlyle", "Leann", "Kelcey", "Laurence", "Ryan", "Reynold", "Mark", "Collyn",
+ "Audie", "Sammy", "Ellery", "Sallie", "Pamelia", "Adolph", "Lydia", "Titania", "Ron", "Bridger", "Aline",
+ "Read", "Kelleigh", "Weldon", "Irving", "Garey", "Diggory", "Evander", "Kylee", "Deidre", "Ormond",
+ "Laurine", "Reannon", "Arline", "Pat"
+
+ };
+
+ public static String[] jargon = { "wireless", "signal", "network", "3G", "plan", "touch-screen",
+ "customer-service", "reachability", "voice-command", "shortcut-menu", "customization", "platform", "speed",
+ "voice-clarity", "voicemail-service" };
+
+ public static String[] vendors = { "at&t", "verizon", "t-mobile", "sprint", "motorola", "samsung", "iphone" };
+
+ public static String[] org_list = { "Latsonity", "ganjalax", "Zuncan", "Lexitechno", "Hot-tech", "subtam",
+ "Coneflex", "Ganjatax", "physcane", "Tranzap", "Qvohouse", "Zununoing", "jaydax", "Keytech", "goldendexon",
+ "Villa-tech", "Trustbam", "Newcom", "Voltlane", "Ontohothex", "Ranhotfan", "Alphadax", "Transhigh",
+ "kin-ron", "Doublezone", "Solophase", "Vivaace", "silfind", "Basecone", "sonstreet", "Freshfix",
+ "Techitechi", "Kanelectrics", "linedexon", "Goldcity", "Newfase", "Technohow", "Zimcone", "Salthex",
+ "U-ron", "Solfix", "whitestreet", "Xx-technology", "Hexviafind", "over-it", "Strongtone", "Tripplelane",
+ "geomedia", "Scotcity", "Inchex", "Vaiatech", "Striptaxon", "Hatcom", "tresline", "Sanjodax", "freshdox",
+ "Sumlane", "Quadlane", "Newphase", "overtech", "Voltbam", "Icerunin", "Fixdintex", "Hexsanhex", "Statcode",
+ "Greencare", "U-electrics", "Zamcorporation", "Ontotanin", "Tanzimcare", "Groovetex", "Ganjastrip",
+ "Redelectronics", "Dandamace", "Whitemedia", "strongex", "Streettax", "highfax", "Mathtech", "Xx-drill",
+ "Sublamdox", "Unijobam", "Rungozoom", "Fixelectrics", "Villa-dox", "Ransaofan", "Plexlane", "itlab",
+ "Lexicone", "Fax-fax", "Viatechi", "Inchdox", "Kongreen", "Doncare", "Y-geohex", "Opeelectronics",
+ "Medflex", "Dancode", "Roundhex", "Labzatron", "Newhotplus", "Sancone", "Ronholdings", "Quoline",
+ "zoomplus", "Fix-touch", "Codetechno", "Tanzumbam", "Indiex", "Canline" };
+}
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGenerator2.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGenerator2.java
new file mode 100644
index 0000000..ad7daaf
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGenerator2.java
@@ -0,0 +1,2483 @@
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+import java.util.UUID;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.OutputKeys;
+import javax.xml.transform.Transformer;
+import javax.xml.transform.TransformerException;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.transform.stream.StreamResult;
+
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+public class DataGenerator2 {
+
+ private RandomDateGenerator randDateGen;
+ private RandomNameGenerator randNameGen;
+ private RandomEmploymentGenerator randEmpGen;
+ private RandomMessageGenerator randMessageGen;
+ private RandomLocationGenerator randLocationGen;
+
+ private DistributionHandler fbDistHandler;
+ private DistributionHandler twDistHandler;
+
+ private int totalFbMessages;
+ private int numFbOnlyUsers;
+ private int totalTwMessages;
+ private int numTwOnlyUsers;
+
+ private int numCommonUsers;
+
+ private int fbUserId;
+ private int twUserId;
+
+ private int fbMessageId;
+ private int twMessageId;
+
+ private Random random = new Random();
+
+ private String commonUserFbSuffix = "_fb";
+ private String commonUserTwSuffix = "_tw";
+
+ private String outputDir;
+
+ private PartitionConfiguration partition;
+
+ private FacebookUser fbUser = new FacebookUser();
+ private TwitterUser twUser = new TwitterUser();
+
+ private FacebookMessage fbMessage = new FacebookMessage();
+ private TweetMessage twMessage = new TweetMessage();
+
+ private int duration;
+
+ private DateFormat dateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss");
+
+ public DataGenerator2(String[] args) throws Exception {
+ String controllerInstallDir = args[0];
+ String partitionConfXML = controllerInstallDir + "/output/partition-conf.xml";
+ String partitionName = args[1];
+ partition = XMLUtil.getPartitionConfiguration(partitionConfXML, partitionName);
+
+ // 1
+ randDateGen = new RandomDateGenerator(new Date(1, 1, 2005), new Date(8, 20, 2012));
+
+ String firstNameFile = controllerInstallDir + "/metadata/firstNames.txt";
+ String lastNameFile = controllerInstallDir + "/metadata/lastNames.txt";
+ String vendorFile = controllerInstallDir + "/metadata/vendors.txt";
+ String jargonFile = controllerInstallDir + "/metadata/jargon.txt";
+ String orgList = controllerInstallDir + "/metadata/org_list.txt";
+
+ randNameGen = new RandomNameGenerator(firstNameFile, lastNameFile);
+ randEmpGen = new RandomEmploymentGenerator(90, new Date(1, 1, 2000), new Date(8, 20, 2012), orgList);
+ randLocationGen = new RandomLocationGenerator(24, 49, 66, 98);
+ randMessageGen = new RandomMessageGenerator(vendorFile, jargonFile);
+
+ totalFbMessages = partition.getTargetPartition().getFbMessageIdMax()
+ - partition.getTargetPartition().getFbMessageIdMin() + 1;
+ numFbOnlyUsers = (partition.getTargetPartition().getFbUserKeyMax()
+ - partition.getTargetPartition().getFbUserKeyMin() + 1)
+ - partition.getTargetPartition().getCommonUsers();
+
+ totalTwMessages = partition.getTargetPartition().getTwMessageIdMax()
+ - partition.getTargetPartition().getTwMessageIdMin() + 1;
+ numTwOnlyUsers = (partition.getTargetPartition().getTwUserKeyMax()
+ - partition.getTargetPartition().getTwUserKeyMin() + 1)
+ - partition.getTargetPartition().getCommonUsers();
+
+ numCommonUsers = partition.getTargetPartition().getCommonUsers();
+ fbDistHandler = new DistributionHandler(totalFbMessages, 0.5, numFbOnlyUsers + numCommonUsers);
+ twDistHandler = new DistributionHandler(totalTwMessages, 0.5, numTwOnlyUsers + numCommonUsers);
+
+ fbUserId = partition.getTargetPartition().getFbUserKeyMin();
+ twUserId = partition.getTargetPartition().getTwUserKeyMin();
+
+ fbMessageId = partition.getTargetPartition().getFbMessageIdMin();
+ twMessageId = partition.getTargetPartition().getTwMessageIdMin();
+
+ outputDir = partition.getSourcePartition().getPath();
+ }
+
+ public DataGenerator2(InitializationInfo info) {
+ initialize(info);
+ }
+
+ private void generateFacebookOnlyUsers(int numFacebookUsers) throws IOException {
+ FileAppender appender = FileUtil.getFileAppender(outputDir + "/" + "fb_users.adm", true, true);
+ FileAppender messageAppender = FileUtil.getFileAppender(outputDir + "/" + "fb_message.adm", true, true);
+
+ for (int i = 0; i < numFacebookUsers; i++) {
+ getFacebookUser(null);
+ appender.appendToFile(fbUser.toString());
+ generateFacebookMessages(fbUser, messageAppender, -1);
+ }
+ appender.close();
+ messageAppender.close();
+ }
+
+ private void generateTwitterOnlyUsers(int numTwitterUsers) throws IOException {
+ FileAppender appender = FileUtil.getFileAppender(outputDir + "/" + "tw_users.adm", true, true);
+ FileAppender messageAppender = FileUtil.getFileAppender(outputDir + "/" + "tw_message.adm", true, true);
+
+ for (int i = 0; i < numTwitterUsers; i++) {
+ getTwitterUser(null);
+ appender.appendToFile(twUser.toString());
+ generateTwitterMessages(twUser, messageAppender, -1);
+ }
+ appender.close();
+ messageAppender.close();
+ }
+
+ private void generateCommonUsers() throws IOException {
+ FileAppender fbAppender = FileUtil.getFileAppender(outputDir + "/" + "fb_users.adm", true, false);
+ FileAppender twAppender = FileUtil.getFileAppender(outputDir + "/" + "tw_users.adm", true, false);
+ FileAppender fbMessageAppender = FileUtil.getFileAppender(outputDir + "/" + "fb_message.adm", true, false);
+ FileAppender twMessageAppender = FileUtil.getFileAppender(outputDir + "/" + "tw_message.adm", true, false);
+
+ for (int i = 0; i < numCommonUsers; i++) {
+ getFacebookUser(commonUserFbSuffix);
+ fbAppender.appendToFile(fbUser.toString());
+ generateFacebookMessages(fbUser, fbMessageAppender, -1);
+
+ getCorrespondingTwitterUser(fbUser);
+ twAppender.appendToFile(twUser.toString());
+ generateTwitterMessages(twUser, twMessageAppender, -1);
+ }
+
+ fbAppender.close();
+ twAppender.close();
+ fbMessageAppender.close();
+ twMessageAppender.close();
+ }
+
+ private void generateFacebookMessages(FacebookUser user, FileAppender appender, int numMsg) throws IOException {
+ Message message;
+ int numMessages = 0;
+ if (numMsg == -1) {
+ numMessages = fbDistHandler
+ .getFromDistribution(fbUserId - partition.getTargetPartition().getFbUserKeyMin());
+ }
+ for (int i = 0; i < numMessages; i++) {
+ message = randMessageGen.getNextRandomMessage();
+ Point location = randLocationGen.getRandomPoint();
+ fbMessage.reset(fbMessageId++, user.getId(), random.nextInt(totalFbMessages + 1), location, message);
+ appender.appendToFile(fbMessage.toString());
+ }
+ }
+
+ private void generateTwitterMessages(TwitterUser user, FileAppender appender, int numMsg) throws IOException {
+ Message message;
+ int numMessages = 0;
+ if (numMsg == -1) {
+ numMessages = twDistHandler
+ .getFromDistribution(twUserId - partition.getTargetPartition().getTwUserKeyMin());
+ }
+
+ for (int i = 0; i < numMessages; i++) {
+ message = randMessageGen.getNextRandomMessage();
+ Point location = randLocationGen.getRandomPoint();
+ DateTime sendTime = randDateGen.getNextRandomDatetime();
+ twMessage.reset(twMessageId + "", user, location, sendTime, message.getReferredTopics(), message);
+ twMessageId++;
+ appender.appendToFile(twMessage.toString());
+ }
+ }
+
+ public Iterator<TweetMessage> getTwitterMessageIterator() {
+ return new TweetMessageIterator(duration);
+ }
+
+ public class TweetMessageIterator implements Iterator<TweetMessage> {
+
+ private final int duration;
+ private long startTime = 0;
+
+ public TweetMessageIterator(int duration) {
+ this.duration = duration;
+ }
+
+ @Override
+ public boolean hasNext() {
+ if (startTime == 0) {
+ startTime = System.currentTimeMillis();
+ }
+ return System.currentTimeMillis() - startTime < duration * 1000;
+ }
+
+ @Override
+ public TweetMessage next() {
+ getTwitterUser(null);
+ Message message = randMessageGen.getNextRandomMessage();
+ Point location = randLocationGen.getRandomPoint();
+ DateTime sendTime = randDateGen.getNextRandomDatetime();
+ twMessage.reset(UUID.randomUUID().toString(), twUser, location, sendTime, message.getReferredTopics(),
+ message);
+ twMessageId++;
+ if (twUserId > numTwOnlyUsers) {
+ twUserId = 1;
+ }
+ return twMessage;
+
+ }
+
+ @Override
+ public void remove() {
+ // TODO Auto-generated method stub
+
+ }
+
+ }
+
+ public static class InitializationInfo {
+ public Date startDate = new Date(1, 1, 2005);
+ public Date endDate = new Date(8, 20, 2012);
+ public String[] lastNames = DataGenerator.lastNames;
+ public String[] firstNames = DataGenerator.firstNames;
+ public String[] vendors = DataGenerator.vendors;
+ public String[] jargon = DataGenerator.jargon;
+ public String[] org_list = DataGenerator.org_list;
+ public int percentEmployed = 90;
+ public Date employmentStartDate = new Date(1, 1, 2000);
+ public Date employmentEndDate = new Date(31, 12, 2012);
+ public int totalFbMessages;
+ public int numFbOnlyUsers;
+ public int totalTwMessages;
+ public int numTwOnlyUsers = 5000;
+ public int numCommonUsers;
+ public int fbUserIdMin;
+ public int fbMessageIdMin;
+ public int twUserIdMin;
+ public int twMessageIdMin;
+ public int timeDurationInSecs = 60;
+
+ }
+
+ public void initialize(InitializationInfo info) {
+ randDateGen = new RandomDateGenerator(info.startDate, info.endDate);
+ randNameGen = new RandomNameGenerator(info.firstNames, info.lastNames);
+ randEmpGen = new RandomEmploymentGenerator(info.percentEmployed, info.employmentStartDate,
+ info.employmentEndDate, info.org_list);
+ randLocationGen = new RandomLocationGenerator(24, 49, 66, 98);
+ randMessageGen = new RandomMessageGenerator(info.vendors, info.jargon);
+ fbDistHandler = new DistributionHandler(info.totalFbMessages, 0.5, info.numFbOnlyUsers + info.numCommonUsers);
+ twDistHandler = new DistributionHandler(info.totalTwMessages, 0.5, info.numTwOnlyUsers + info.numCommonUsers);
+ fbUserId = info.fbUserIdMin;
+ twUserId = info.twUserIdMin;
+
+ fbMessageId = info.fbMessageIdMin;
+ twMessageId = info.fbMessageIdMin;
+ duration = info.timeDurationInSecs;
+ }
+
+ public static void main(String args[]) throws Exception {
+
+ String controllerInstallDir = null;
+ if (args.length < 2) {
+ printUsage();
+ System.exit(1);
+ }
+
+ DataGenerator2 dataGenerator = new DataGenerator2(args);
+ dataGenerator.generateData();
+ }
+
+ public static void printUsage() {
+ System.out.println(" Error: Invalid number of arguments ");
+ System.out.println(" Usage :" + " DataGenerator <path to configuration file> <partition name> ");
+ }
+
+ public void generateData() throws IOException {
+ generateFacebookOnlyUsers(numFbOnlyUsers);
+ generateTwitterOnlyUsers(numTwOnlyUsers);
+ generateCommonUsers();
+ System.out.println("Partition :" + partition.getTargetPartition().getName() + " finished");
+ }
+
+ public void getFacebookUser(String usernameSuffix) {
+ String suggestedName = randNameGen.getRandomName();
+ String[] nameComponents = suggestedName.split(" ");
+ String name = nameComponents[0] + nameComponents[1];
+ if (usernameSuffix != null) {
+ name = name + usernameSuffix;
+ }
+ String alias = nameComponents[0];
+ String userSince = randDateGen.getNextRandomDatetime().toString();
+ int numFriends = random.nextInt(25);
+ int[] friendIds = RandomUtil.getKFromN(numFriends, (numFbOnlyUsers + numCommonUsers));
+ Employment emp = randEmpGen.getRandomEmployment();
+ fbUser.reset(fbUserId++, alias, name, userSince, friendIds, emp);
+ }
+
+ public void getTwitterUser(String usernameSuffix) {
+ String suggestedName = randNameGen.getRandomName();
+ String[] nameComponents = suggestedName.split(" ");
+ String screenName = nameComponents[0] + nameComponents[1] + randNameGen.getRandomNameSuffix();
+ String name = suggestedName;
+ if (usernameSuffix != null) {
+ name = name + usernameSuffix;
+ }
+ int numFriends = random.nextInt((int) (100)); // draw from Zipfian
+ int statusesCount = random.nextInt(500); // draw from Zipfian
+ int followersCount = random.nextInt((int) (200));
+ twUser.reset(screenName, numFriends, statusesCount, name, followersCount);
+ twUserId++;
+ }
+
+ public void getCorrespondingTwitterUser(FacebookUser fbUser) {
+ String screenName = fbUser.getName().substring(0, fbUser.getName().lastIndexOf(commonUserFbSuffix))
+ + commonUserTwSuffix;
+ String name = screenName.split(" ")[0];
+ int numFriends = random.nextInt((int) ((numTwOnlyUsers + numCommonUsers)));
+ int statusesCount = random.nextInt(500); // draw from Zipfian
+ int followersCount = random.nextInt((int) (numTwOnlyUsers + numCommonUsers));
+ twUser.reset(screenName, numFriends, statusesCount, name, followersCount);
+ }
+
+ public static class RandomDateGenerator {
+
+ private final Date startDate;
+ private final Date endDate;
+ private final Random random = new Random();
+ private final int yearDifference;
+ private Date workingDate;
+ private Date recentDate;
+ private DateTime dateTime;
+
+ public RandomDateGenerator(Date startDate, Date endDate) {
+ this.startDate = startDate;
+ this.endDate = endDate;
+ yearDifference = endDate.getYear() - startDate.getYear() + 1;
+ workingDate = new Date();
+ recentDate = new Date();
+ dateTime = new DateTime();
+ }
+
+ public Date getStartDate() {
+ return startDate;
+ }
+
+ public Date getEndDate() {
+ return endDate;
+ }
+
+ public Date getNextRandomDate() {
+ int year = random.nextInt(yearDifference) + startDate.getYear();
+ int month;
+ int day;
+ if (year == endDate.getYear()) {
+ month = random.nextInt(endDate.getMonth()) + 1;
+ if (month == endDate.getMonth()) {
+ day = random.nextInt(endDate.getDay()) + 1;
+ } else {
+ day = random.nextInt(28) + 1;
+ }
+ } else {
+ month = random.nextInt(12) + 1;
+ day = random.nextInt(28) + 1;
+ }
+ workingDate.reset(month, day, year);
+ return workingDate;
+ }
+
+ public DateTime getNextRandomDatetime() {
+ Date randomDate = getNextRandomDate();
+ dateTime.reset(randomDate);
+ return dateTime;
+ }
+
+ public Date getNextRecentDate(Date date) {
+ int year = date.getYear()
+ + (date.getYear() == endDate.getYear() ? 0 : random.nextInt(endDate.getYear() - date.getYear()));
+ int month = (year == endDate.getYear()) ? date.getMonth() == endDate.getMonth() ? (endDate.getMonth())
+ : (date.getMonth() + random.nextInt(endDate.getMonth() - date.getMonth())) : random.nextInt(12) + 1;
+
+ int day = (year == endDate.getYear()) ? month == endDate.getMonth() ? date.getDay() == endDate.getDay() ? endDate
+ .getDay() : date.getDay() + random.nextInt(endDate.getDay() - date.getDay())
+ : random.nextInt(28) + 1
+ : random.nextInt(28) + 1;
+ recentDate.reset(month, day, year);
+ return recentDate;
+ }
+
+ public static void main(String args[]) throws Exception {
+ Date date = new Date(2, 20, 2012);
+ RandomDateGenerator dgen = new RandomDateGenerator(new Date(1, 1, 2005), new Date(8, 20, 2012));
+ while (true) {
+ Date nextDate = dgen.getNextRandomDate();
+ if (nextDate.getDay() == 0) {
+ throw new Exception("invalid date " + nextDate);
+ }
+
+ // System.out.println(" original date: " + date);
+ System.out.println(nextDate);
+ }
+ }
+ }
+
+ public static class DateTime extends Date {
+
+ private String hour = "10";
+ private String min = "10";
+ private String sec = "00";
+ private long chrononTime;
+
+ public DateTime(int month, int day, int year, String hour, String min, String sec) {
+ super(month, day, year);
+ this.hour = hour;
+ this.min = min;
+ this.sec = sec;
+ chrononTime = new java.util.Date(year, month, day, Integer.parseInt(hour), Integer.parseInt(min),
+ Integer.parseInt(sec)).getTime();
+ }
+
+ public void reset(int month, int day, int year, String hour, String min, String sec) {
+ super.setDay(month);
+ super.setDay(day);
+ super.setYear(year);
+ this.hour = hour;
+ this.min = min;
+ this.sec = sec;
+ chrononTime = new java.util.Date(year, month, day, Integer.parseInt(hour), Integer.parseInt(min),
+ Integer.parseInt(sec)).getTime();
+ }
+
+ public DateTime() {
+ }
+
+ public DateTime(Date date) {
+ super(date.getMonth(), date.getDay(), date.getYear());
+ }
+
+ public void reset(Date date) {
+ reset(date.getMonth(), date.getDay(), date.getYear());
+ }
+
+ public DateTime(Date date, int hour, int min, int sec) {
+ super(date.getMonth(), date.getDay(), date.getYear());
+ this.hour = (hour < 10) ? "0" : "" + hour;
+ this.min = (min < 10) ? "0" : "" + min;
+ this.sec = (sec < 10) ? "0" : "" + sec;
+ }
+
+ public long getChrononTime() {
+ return chrononTime;
+ }
+
+ public String getHour() {
+ return hour;
+ }
+
+ public String getMin() {
+ return min;
+ }
+
+ public String getSec() {
+ return sec;
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("datetime");
+ builder.append("(\"");
+ builder.append(super.getYear());
+ builder.append("-");
+ builder.append(super.getMonth() < 10 ? "0" + super.getMonth() : super.getMonth());
+ builder.append("-");
+ builder.append(super.getDay() < 10 ? "0" + super.getDay() : super.getDay());
+ builder.append("T");
+ builder.append(hour + ":" + min + ":" + sec);
+ builder.append("\")");
+ return builder.toString();
+ }
+ }
+
+ public static class Message {
+
+ private char[] message = new char[500];
+ private List<String> referredTopics;
+ private int length;
+
+ public Message(char[] m, List<String> referredTopics) {
+ System.arraycopy(m, 0, message, 0, m.length);
+ length = m.length;
+ this.referredTopics = referredTopics;
+ }
+
+ public Message() {
+ referredTopics = new ArrayList<String>();
+ length = 0;
+ }
+
+ public char[] getMessage() {
+ return message;
+ }
+
+ public List<String> getReferredTopics() {
+ return referredTopics;
+ }
+
+ public void reset(char[] m, int offset, int length, List<String> referredTopics) {
+ System.arraycopy(m, offset, message, 0, length);
+ this.length = length;
+ this.referredTopics = referredTopics;
+ }
+
+ public int getLength() {
+ return length;
+ }
+
+ public char charAt(int index) {
+ return message[index];
+ }
+
+ }
+
+ public static class Point {
+
+ private float latitude;
+ private float longitude;
+
+ public float getLatitude() {
+ return latitude;
+ }
+
+ public float getLongitude() {
+ return longitude;
+ }
+
+ public Point(float latitude, float longitude) {
+ this.latitude = latitude;
+ this.longitude = longitude;
+ }
+
+ public void reset(float latitude, float longitude) {
+ this.latitude = latitude;
+ this.longitude = longitude;
+ }
+
+ public Point() {
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("point(\"" + latitude + "," + longitude + "\")");
+ return builder.toString();
+ }
+ }
+
+ public static class RandomNameGenerator {
+
+ private String[] firstNames;
+ private String[] lastNames;
+
+ private final Random random = new Random();
+
+ private final String[] connectors = new String[] { "_", "#", "$", "@" };
+
+ public RandomNameGenerator(String firstNameFilePath, String lastNameFilePath) throws IOException {
+ firstNames = FileUtil.listyFile(new File(firstNameFilePath)).toArray(new String[] {});
+ lastNames = FileUtil.listyFile(new File(lastNameFilePath)).toArray(new String[] {});
+ }
+
+ public RandomNameGenerator(String[] firstNames, String[] lastNames) {
+ this.firstNames = firstNames;
+ this.lastNames = lastNames;
+ }
+
+ public String getRandomName() {
+ String name;
+ name = getSuggestedName();
+ return name;
+
+ }
+
+ private String getSuggestedName() {
+ int firstNameIndex = random.nextInt(firstNames.length);
+ int lastNameIndex = random.nextInt(lastNames.length);
+ String suggestedName = firstNames[firstNameIndex] + " " + lastNames[lastNameIndex];
+ return suggestedName;
+ }
+
+ public String getRandomNameSuffix() {
+ return connectors[random.nextInt(connectors.length)] + random.nextInt(1000);
+ }
+ }
+
+ public static class RandomMessageGenerator {
+
+ private final MessageTemplate messageTemplate;
+
+ public RandomMessageGenerator(String vendorFilePath, String jargonFilePath) throws IOException {
+ List<String> vendors = FileUtil.listyFile(new File(vendorFilePath));
+ List<String> jargon = FileUtil.listyFile(new File(jargonFilePath));
+ this.messageTemplate = new MessageTemplate(vendors, jargon);
+ }
+
+ public RandomMessageGenerator(String[] vendors, String[] jargon) {
+ List<String> vendorList = new ArrayList<String>();
+ for (String v : vendors) {
+ vendorList.add(v);
+ }
+ List<String> jargonList = new ArrayList<String>();
+ for (String j : jargon) {
+ jargonList.add(j);
+ }
+ this.messageTemplate = new MessageTemplate(vendorList, jargonList);
+ }
+
+ public Message getNextRandomMessage() {
+ return messageTemplate.getNextMessage();
+ }
+ }
+
+ public static class AbstractMessageTemplate {
+
+ protected final Random random = new Random();
+
+ protected String[] positiveVerbs = new String[] { "like", "love" };
+ protected String[] negativeVerbs = new String[] { "dislike", "hate", "can't stand" };
+
+ protected String[] negativeAdjectives = new String[] { "horrible", "bad", "terrible", "OMG" };
+ protected String[] postiveAdjectives = new String[] { "good", "awesome", "amazing", "mind-blowing" };
+
+ protected String[] otherWords = new String[] { "the", "its" };
+ }
+
+ public static class MessageTemplate extends AbstractMessageTemplate {
+
+ private List<String> vendors;
+ private List<String> jargon;
+ private CharBuffer buffer;
+ private List<String> referredTopics;
+ private Message message = new Message();
+
+ public MessageTemplate(List<String> vendors, List<String> jargon) {
+ this.vendors = vendors;
+ this.jargon = jargon;
+ buffer = CharBuffer.allocate(2500);
+ referredTopics = new ArrayList<String>();
+ }
+
+ public Message getNextMessage() {
+ buffer.position(0);
+ buffer.limit(2500);
+ referredTopics.clear();
+ boolean isPositive = random.nextBoolean();
+ String[] verbArray = isPositive ? positiveVerbs : negativeVerbs;
+ String[] adjectiveArray = isPositive ? postiveAdjectives : negativeAdjectives;
+ String verb = verbArray[random.nextInt(verbArray.length)];
+ String adjective = adjectiveArray[random.nextInt(adjectiveArray.length)];
+
+ buffer.put(" ");
+ buffer.put(verb);
+ buffer.put(" ");
+ String vendor = vendors.get(random.nextInt(vendors.size()));
+ referredTopics.add(vendor);
+ buffer.append(vendor);
+ buffer.append(" ");
+ buffer.append(otherWords[random.nextInt(otherWords.length)]);
+ buffer.append(" ");
+ String jargonTerm = jargon.get(random.nextInt(jargon.size()));
+ referredTopics.add(jargonTerm);
+ buffer.append(jargonTerm);
+ buffer.append(" is ");
+ buffer.append(adjective);
+ if (random.nextBoolean()) {
+ buffer.append(isPositive ? ":)" : ":(");
+ }
+
+ buffer.flip();
+ message.reset(buffer.array(), 0, buffer.limit(), referredTopics);
+ return message;
+ }
+ }
+
+ public static class RandomUtil {
+
+ public static Random random = new Random();
+
+ public static int[] getKFromN(int k, int n) {
+ int[] result = new int[k];
+ int cnt = 0;
+ HashSet<Integer> values = new HashSet<Integer>();
+ while (cnt < k) {
+ int val = random.nextInt(n + 1);
+ if (values.contains(val)) {
+ continue;
+ }
+
+ result[cnt++] = val;
+ values.add(val);
+ }
+ return result;
+ }
+ }
+
+ public static class FileUtil {
+
+ public static List<String> listyFile(File file) throws IOException {
+
+ BufferedReader reader = new BufferedReader(new FileReader(file));
+ String line;
+ List<String> list = new ArrayList<String>();
+ while (true) {
+ line = reader.readLine();
+ if (line == null) {
+ break;
+ }
+ list.add(line);
+ }
+ return list;
+ }
+
+ public static FileAppender getFileAppender(String filePath, boolean createIfNotExists, boolean overwrite)
+ throws IOException {
+ return new FileAppender(filePath, createIfNotExists, overwrite);
+ }
+ }
+
+ public static class FileAppender {
+
+ private final BufferedWriter writer;
+
+ public FileAppender(String filePath, boolean createIfNotExists, boolean overwrite) throws IOException {
+ File file = new File(filePath);
+ if (!file.exists()) {
+ if (createIfNotExists) {
+ new File(file.getParent()).mkdirs();
+ } else {
+ throw new IOException("path " + filePath + " does not exists");
+ }
+ }
+ this.writer = new BufferedWriter(new FileWriter(file, !overwrite));
+ }
+
+ public void appendToFile(String content) throws IOException {
+ writer.append(content);
+ writer.append("\n");
+ }
+
+ public void close() throws IOException {
+ writer.close();
+ }
+ }
+
+ public static class RandomEmploymentGenerator {
+
+ private final int percentEmployed;
+ private final Random random = new Random();
+ private final RandomDateGenerator randDateGen;
+ private final List<String> organizations;
+ private Employment emp;
+
+ public RandomEmploymentGenerator(int percentEmployed, Date beginEmpDate, Date endEmpDate, String orgListPath)
+ throws IOException {
+ this.percentEmployed = percentEmployed;
+ this.randDateGen = new RandomDateGenerator(beginEmpDate, endEmpDate);
+ organizations = FileUtil.listyFile(new File(orgListPath));
+ emp = new Employment();
+ }
+
+ public RandomEmploymentGenerator(int percentEmployed, Date beginEmpDate, Date endEmpDate, String[] orgList) {
+ this.percentEmployed = percentEmployed;
+ this.randDateGen = new RandomDateGenerator(beginEmpDate, endEmpDate);
+ organizations = new ArrayList<String>();
+ for (String org : orgList) {
+ organizations.add(org);
+ }
+ emp = new Employment();
+ }
+
+ public Employment getRandomEmployment() {
+ int empployed = random.nextInt(100) + 1;
+ boolean isEmployed = false;
+ if (empployed <= percentEmployed) {
+ isEmployed = true;
+ }
+ Date startDate = randDateGen.getNextRandomDate();
+ Date endDate = null;
+ if (!isEmployed) {
+ endDate = randDateGen.getNextRecentDate(startDate);
+ }
+ String org = organizations.get(random.nextInt(organizations.size()));
+ emp.reset(org, startDate, endDate);
+ return emp;
+ }
+ }
+
+ public static class RandomLocationGenerator {
+
+ private Random random = new Random();
+
+ private final int beginLat;
+ private final int endLat;
+ private final int beginLong;
+ private final int endLong;
+
+ private Point point;
+
+ public RandomLocationGenerator(int beginLat, int endLat, int beginLong, int endLong) {
+ this.beginLat = beginLat;
+ this.endLat = endLat;
+ this.beginLong = beginLong;
+ this.endLong = endLong;
+ this.point = new Point();
+ }
+
+ public Point getRandomPoint() {
+ int latMajor = beginLat + random.nextInt(endLat - beginLat);
+ int latMinor = random.nextInt(100);
+ float latitude = latMajor + ((float) latMinor) / 100;
+
+ int longMajor = beginLong + random.nextInt(endLong - beginLong);
+ int longMinor = random.nextInt(100);
+ float longitude = longMajor + ((float) longMinor) / 100;
+
+ point.reset(latitude, longitude);
+ return point;
+ }
+
+ }
+
+ public static class PartitionConfiguration {
+
+ private final TargetPartition targetPartition;
+ private final SourcePartition sourcePartition;
+
+ public PartitionConfiguration(SourcePartition sourcePartition, TargetPartition targetPartition) {
+ this.sourcePartition = sourcePartition;
+ this.targetPartition = targetPartition;
+ }
+
+ public TargetPartition getTargetPartition() {
+ return targetPartition;
+ }
+
+ public SourcePartition getSourcePartition() {
+ return sourcePartition;
+ }
+
+ }
+
+ public static class SourcePartition {
+
+ private final String name;
+ private final String host;
+ private final String path;
+
+ public SourcePartition(String name, String host, String path) {
+ this.name = name;
+ this.host = host;
+ this.path = path;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public String getHost() {
+ return host;
+ }
+
+ public String getPath() {
+ return path;
+ }
+ }
+
+ public static class TargetPartition {
+ private final String name;
+ private final String host;
+ private final String path;
+ private final int fbUserKeyMin;
+ private final int fbUserKeyMax;
+ private final int twUserKeyMin;
+ private final int twUserKeyMax;
+ private final int fbMessageIdMin;
+ private final int fbMessageIdMax;
+ private final int twMessageIdMin;
+ private final int twMessageIdMax;
+ private final int commonUsers;
+
+ public TargetPartition(String partitionName, String host, String path, int fbUserKeyMin, int fbUserKeyMax,
+ int twUserKeyMin, int twUserKeyMax, int fbMessageIdMin, int fbMessageIdMax, int twMessageIdMin,
+ int twMessageIdMax, int commonUsers) {
+ this.name = partitionName;
+ this.host = host;
+ this.path = path;
+ this.fbUserKeyMin = fbUserKeyMin;
+ this.fbUserKeyMax = fbUserKeyMax;
+ this.twUserKeyMin = twUserKeyMin;
+ this.twUserKeyMax = twUserKeyMax;
+ this.twMessageIdMin = twMessageIdMin;
+ this.twMessageIdMax = twMessageIdMax;
+ this.fbMessageIdMin = fbMessageIdMin;
+ this.fbMessageIdMax = fbMessageIdMax;
+ this.commonUsers = commonUsers;
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append(name);
+ builder.append(" ");
+ builder.append(host);
+ builder.append("\n");
+ builder.append(path);
+ builder.append("\n");
+ builder.append("fbUser:key:min");
+ builder.append(fbUserKeyMin);
+
+ builder.append("\n");
+ builder.append("fbUser:key:max");
+ builder.append(fbUserKeyMax);
+
+ builder.append("\n");
+ builder.append("twUser:key:min");
+ builder.append(twUserKeyMin);
+
+ builder.append("\n");
+ builder.append("twUser:key:max");
+ builder.append(twUserKeyMax);
+
+ builder.append("\n");
+ builder.append("fbMessage:key:min");
+ builder.append(fbMessageIdMin);
+
+ builder.append("\n");
+ builder.append("fbMessage:key:max");
+ builder.append(fbMessageIdMax);
+
+ builder.append("\n");
+ builder.append("twMessage:key:min");
+ builder.append(twMessageIdMin);
+
+ builder.append("\n");
+ builder.append("twMessage:key:max");
+ builder.append(twMessageIdMax);
+
+ builder.append("\n");
+ builder.append("twMessage:key:max");
+ builder.append(twMessageIdMax);
+
+ builder.append("\n");
+ builder.append("commonUsers");
+ builder.append(commonUsers);
+
+ return new String(builder);
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public String getHost() {
+ return host;
+ }
+
+ public int getFbUserKeyMin() {
+ return fbUserKeyMin;
+ }
+
+ public int getFbUserKeyMax() {
+ return fbUserKeyMax;
+ }
+
+ public int getTwUserKeyMin() {
+ return twUserKeyMin;
+ }
+
+ public int getTwUserKeyMax() {
+ return twUserKeyMax;
+ }
+
+ public int getFbMessageIdMin() {
+ return fbMessageIdMin;
+ }
+
+ public int getFbMessageIdMax() {
+ return fbMessageIdMax;
+ }
+
+ public int getTwMessageIdMin() {
+ return twMessageIdMin;
+ }
+
+ public int getTwMessageIdMax() {
+ return twMessageIdMax;
+ }
+
+ public int getCommonUsers() {
+ return commonUsers;
+ }
+
+ public String getPath() {
+ return path;
+ }
+ }
+
+ public static class Employment {
+
+ private String organization;
+ private Date startDate;
+ private Date endDate;
+
+ public Employment(String organization, Date startDate, Date endDate) {
+ this.organization = organization;
+ this.startDate = startDate;
+ this.endDate = endDate;
+ }
+
+ public Employment() {
+ }
+
+ public void reset(String organization, Date startDate, Date endDate) {
+ this.organization = organization;
+ this.startDate = startDate;
+ this.endDate = endDate;
+ }
+
+ public String getOrganization() {
+ return organization;
+ }
+
+ public Date getStartDate() {
+ return startDate;
+ }
+
+ public Date getEndDate() {
+ return endDate;
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder("");
+ builder.append("{");
+ builder.append("\"organization-name\":");
+ builder.append("\"" + organization + "\"");
+ builder.append(",");
+ builder.append("\"start-date\":");
+ builder.append(startDate);
+ if (endDate != null) {
+ builder.append(",");
+ builder.append("\"end-date\":");
+ builder.append(endDate);
+ }
+ builder.append("}");
+ return new String(builder);
+ }
+
+ }
+
+ public static class FacebookMessage {
+
+ private int messageId;
+ private int authorId;
+ private int inResponseTo;
+ private Point senderLocation;
+ private Message message;
+
+ public int getMessageId() {
+ return messageId;
+ }
+
+ public int getAuthorID() {
+ return authorId;
+ }
+
+ public Point getSenderLocation() {
+ return senderLocation;
+ }
+
+ public Message getMessage() {
+ return message;
+ }
+
+ public int getInResponseTo() {
+ return inResponseTo;
+ }
+
+ public FacebookMessage() {
+
+ }
+
+ public FacebookMessage(int messageId, int authorId, int inResponseTo, Point senderLocation, Message message) {
+ this.messageId = messageId;
+ this.authorId = authorId;
+ this.inResponseTo = inResponseTo;
+ this.senderLocation = senderLocation;
+ this.message = message;
+ }
+
+ public void reset(int messageId, int authorId, int inResponseTo, Point senderLocation, Message message) {
+ this.messageId = messageId;
+ this.authorId = authorId;
+ this.inResponseTo = inResponseTo;
+ this.senderLocation = senderLocation;
+ this.message = message;
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("{");
+ builder.append("\"message-id\":");
+ builder.append(messageId);
+ builder.append(",");
+ builder.append("\"author-id\":");
+ builder.append(authorId);
+ builder.append(",");
+ builder.append("\"in-response-to\":");
+ builder.append(inResponseTo);
+ builder.append(",");
+ builder.append("\"sender-location\":");
+ builder.append(senderLocation);
+ builder.append(",");
+ builder.append("\"message\":");
+ builder.append("\"");
+ for (int i = 0; i < message.getLength(); i++) {
+ builder.append(message.charAt(i));
+ }
+ builder.append("\"");
+ builder.append("}");
+ return new String(builder);
+ }
+ }
+
+ public static class FacebookUser {
+
+ private int id;
+ private String alias;
+ private String name;
+ private String userSince;
+ private int[] friendIds;
+ private Employment employment;
+
+ public FacebookUser() {
+
+ }
+
+ public FacebookUser(int id, String alias, String name, String userSince, int[] friendIds, Employment employment) {
+ this.id = id;
+ this.alias = alias;
+ this.name = name;
+ this.userSince = userSince;
+ this.friendIds = friendIds;
+ this.employment = employment;
+ }
+
+ public int getId() {
+ return id;
+ }
+
+ public String getAlias() {
+ return alias;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public String getUserSince() {
+ return userSince;
+ }
+
+ public int[] getFriendIds() {
+ return friendIds;
+ }
+
+ public Employment getEmployment() {
+ return employment;
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("{");
+ builder.append("\"id\":" + id);
+ builder.append(",");
+ builder.append("\"alias\":" + "\"" + alias + "\"");
+ builder.append(",");
+ builder.append("\"name\":" + "\"" + name + "\"");
+ builder.append(",");
+ builder.append("\"user-since\":" + userSince);
+ builder.append(",");
+ builder.append("\"friend-ids\":");
+ builder.append("{{");
+ for (int i = 0; i < friendIds.length; i++) {
+ builder.append(friendIds[i]);
+ builder.append(",");
+ }
+ if (friendIds.length > 0) {
+ builder.deleteCharAt(builder.lastIndexOf(","));
+ }
+ builder.append("}}");
+ builder.append(",");
+ builder.append("\"employment\":");
+ builder.append("[");
+ builder.append(employment.toString());
+ builder.append("]");
+ builder.append("}");
+ return builder.toString();
+ }
+
+ public void setId(int id) {
+ this.id = id;
+ }
+
+ public void setAlias(String alias) {
+ this.alias = alias;
+ }
+
+ public void setName(String name) {
+ this.name = name;
+ }
+
+ public void setUserSince(String userSince) {
+ this.userSince = userSince;
+ }
+
+ public void setFriendIds(int[] friendIds) {
+ this.friendIds = friendIds;
+ }
+
+ public void setEmployment(Employment employment) {
+ this.employment = employment;
+ }
+
+ public void reset(int id, String alias, String name, String userSince, int[] friendIds, Employment employment) {
+ this.id = id;
+ this.alias = alias;
+ this.name = name;
+ this.userSince = userSince;
+ this.friendIds = friendIds;
+ this.employment = employment;
+ }
+ }
+
+ public static class TweetMessage {
+
+ private String tweetid;
+ private TwitterUser user;
+ private Point senderLocation;
+ private DateTime sendTime;
+ private List<String> referredTopics;
+ private Message messageText;
+
+ public TweetMessage() {
+
+ }
+
+ public TweetMessage(String tweetid, TwitterUser user, Point senderLocation, DateTime sendTime,
+ List<String> referredTopics, Message messageText) {
+ this.tweetid = tweetid;
+ this.user = user;
+ this.senderLocation = senderLocation;
+ this.sendTime = sendTime;
+ this.referredTopics = referredTopics;
+ this.messageText = messageText;
+ }
+
+ public void reset(String tweetid, TwitterUser user, Point senderLocation, DateTime sendTime,
+ List<String> referredTopics, Message messageText) {
+ this.tweetid = tweetid;
+ this.user = user;
+ this.senderLocation = senderLocation;
+ this.sendTime = sendTime;
+ this.referredTopics = referredTopics;
+ this.messageText = messageText;
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("{");
+ builder.append("\"tweetid\":");
+ builder.append("\"" + tweetid + "\"");
+ builder.append(",");
+ builder.append("\"user\":");
+ builder.append(user);
+ builder.append(",");
+ builder.append("\"sender-location\":");
+ builder.append(senderLocation);
+ builder.append(",");
+ builder.append("\"send-time\":");
+ builder.append(sendTime);
+ builder.append(",");
+ builder.append("\"referred-topics\":");
+ builder.append("{{");
+ for (String topic : referredTopics) {
+ builder.append("\"" + topic + "\"");
+ builder.append(",");
+ }
+ if (referredTopics.size() > 0) {
+ builder.deleteCharAt(builder.lastIndexOf(","));
+ }
+ builder.append("}}");
+ builder.append(",");
+ builder.append("\"message-text\":");
+ builder.append("\"");
+ for (int i = 0; i < messageText.getLength(); i++) {
+ builder.append(messageText.charAt(i));
+ }
+ builder.append("\"");
+ builder.append("}");
+ return new String(builder);
+ }
+
+ public String getTweetid() {
+ return tweetid;
+ }
+
+ public void setTweetid(String tweetid) {
+ this.tweetid = tweetid;
+ }
+
+ public TwitterUser getUser() {
+ return user;
+ }
+
+ public void setUser(TwitterUser user) {
+ this.user = user;
+ }
+
+ public Point getSenderLocation() {
+ return senderLocation;
+ }
+
+ public void setSenderLocation(Point senderLocation) {
+ this.senderLocation = senderLocation;
+ }
+
+ public DateTime getSendTime() {
+ return sendTime;
+ }
+
+ public void setSendTime(DateTime sendTime) {
+ this.sendTime = sendTime;
+ }
+
+ public List<String> getReferredTopics() {
+ return referredTopics;
+ }
+
+ public void setReferredTopics(List<String> referredTopics) {
+ this.referredTopics = referredTopics;
+ }
+
+ public Message getMessageText() {
+ return messageText;
+ }
+
+ public void setMessageText(Message messageText) {
+ this.messageText = messageText;
+ }
+
+ }
+
+ public static class TwitterUser {
+
+ private String screenName;
+ private String lang = "en";
+ private int friendsCount;
+ private int statusesCount;
+ private String name;
+ private int followersCount;
+
+ public TwitterUser() {
+
+ }
+
+ public TwitterUser(String screenName, int friendsCount, int statusesCount, String name, int followersCount) {
+ this.screenName = screenName;
+ this.friendsCount = friendsCount;
+ this.statusesCount = statusesCount;
+ this.name = name;
+ this.followersCount = followersCount;
+ }
+
+ public void reset(String screenName, int friendsCount, int statusesCount, String name, int followersCount) {
+ this.screenName = screenName;
+ this.friendsCount = friendsCount;
+ this.statusesCount = statusesCount;
+ this.name = name;
+ this.followersCount = followersCount;
+ }
+
+ public String getScreenName() {
+ return screenName;
+ }
+
+ public int getFriendsCount() {
+ return friendsCount;
+ }
+
+ public int getStatusesCount() {
+ return statusesCount;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public int getFollowersCount() {
+ return followersCount;
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("{");
+ builder.append("\"screen-name\":" + "\"" + screenName + "\"");
+ builder.append(",");
+ builder.append("\"lang\":" + "\"" + lang + "\"");
+ builder.append(",");
+ builder.append("\"friends_count\":" + friendsCount);
+ builder.append(",");
+ builder.append("\"statuses_count\":" + statusesCount);
+ builder.append(",");
+ builder.append("\"name\":" + "\"" + name + "\"");
+ builder.append(",");
+ builder.append("\"followers_count\":" + followersCount);
+ builder.append("}");
+ return builder.toString();
+ }
+
+ }
+
+ public static class DistributionHandler {
+
+ private final ZipfGenerator zipfGen;
+ private final int totalUsers;
+ private final int totalMessages;
+ private Random random = new Random();
+
+ public DistributionHandler(int totalMessages, double skew, int totalNumUsers) {
+ zipfGen = new ZipfGenerator(totalMessages, skew);
+ totalUsers = totalNumUsers;
+ this.totalMessages = totalMessages;
+ }
+
+ public int getFromDistribution(int rank) {
+ double prob = zipfGen.getProbability(rank);
+ int numMessages = (int) (prob * totalMessages);
+ return numMessages;
+ }
+
+ public static void main(String args[]) {
+ int totalMessages = 1000 * 4070;
+ double skew = 0.5;
+ int totalUsers = 4070;
+ DistributionHandler d = new DistributionHandler(totalMessages, skew, totalUsers);
+ int sum = 0;
+ for (int i = totalUsers; i >= 1; i--) {
+ float contrib = d.getFromDistribution(i);
+ sum += contrib;
+ System.out.println(i + ":" + contrib);
+ }
+
+ System.out.println("SUM" + ":" + sum);
+
+ }
+ }
+
+ public static class ZipfGenerator {
+
+ private Random rnd = new Random(System.currentTimeMillis());
+ private int size;
+ private double skew;
+ private double bottom = 0;
+
+ public ZipfGenerator(int size, double skew) {
+ this.size = size;
+ this.skew = skew;
+ for (int i = 1; i < size; i++) {
+ this.bottom += (1 / Math.pow(i, this.skew));
+ }
+ }
+
+ // the next() method returns an rank id. The frequency of returned rank
+ // ids are follows Zipf distribution.
+ public int next() {
+ int rank;
+ double friquency = 0;
+ double dice;
+ rank = rnd.nextInt(size);
+ friquency = (1.0d / Math.pow(rank, this.skew)) / this.bottom;
+ dice = rnd.nextDouble();
+ while (!(dice < friquency)) {
+ rank = rnd.nextInt(size);
+ friquency = (1.0d / Math.pow(rank, this.skew)) / this.bottom;
+ dice = rnd.nextDouble();
+ }
+ return rank;
+ }
+
+ // This method returns a probability that the given rank occurs.
+ public double getProbability(int rank) {
+ return (1.0d / Math.pow(rank, this.skew)) / this.bottom;
+ }
+
+ public static void main(String[] args) throws IOException {
+ int total = (int) (3.7 * 1000 * 1000);
+ int skew = 2;
+ int numUsers = 1000 * 1000;
+ /*
+ * if (args.length != 2) { System.out.println("usage:" +
+ * "./zipf size skew"); System.exit(-1); }
+ */
+ BufferedWriter buf = new BufferedWriter(new FileWriter(new File("/tmp/zip_output")));
+ ZipfGenerator zipf = new ZipfGenerator(total, skew);
+ double sum = 0;
+ for (int i = 1; i <= numUsers; i++) {
+ double prob = zipf.getProbability(i);
+ double contribution = (double) (prob * total);
+ String contrib = i + ":" + contribution;
+ buf.write(contrib);
+ buf.write("\n");
+ System.out.println(contrib);
+ sum += contribution;
+ }
+ System.out.println("sum is :" + sum);
+ }
+ }
+
+ public static class PartitionElement implements ILibraryElement {
+ private final String name;
+ private final String host;
+ private final int fbUserKeyMin;
+ private final int fbUserKeyMax;
+ private final int twUserKeyMin;
+ private final int twUserKeyMax;
+ private final int fbMessageIdMin;
+ private final int fbMessageIdMax;
+ private final int twMessageIdMin;
+ private final int twMessageIdMax;
+
+ public PartitionElement(String partitionName, String host, int fbUserKeyMin, int fbUserKeyMax,
+ int twUserKeyMin, int twUserKeyMax, int fbMessageIdMin, int fbMessageIdMax, int twMessageIdMin,
+ int twMessageIdMax) {
+ this.name = partitionName;
+ this.host = host;
+ this.fbUserKeyMin = fbUserKeyMin;
+ this.fbUserKeyMax = fbUserKeyMax;
+ this.twUserKeyMin = twUserKeyMax;
+ this.twUserKeyMax = twUserKeyMax;
+ this.twMessageIdMin = twMessageIdMin;
+ this.twMessageIdMax = twMessageIdMax;
+ this.fbMessageIdMin = fbMessageIdMin;
+ this.fbMessageIdMax = fbMessageIdMax;
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append(name);
+ builder.append(" ");
+ builder.append(host);
+ builder.append("\n");
+ builder.append("fbUser:key:min");
+ builder.append(fbUserKeyMin);
+
+ builder.append("\n");
+ builder.append("fbUser:key:max");
+ builder.append(fbUserKeyMax);
+
+ builder.append("\n");
+ builder.append("twUser:key:min");
+ builder.append(twUserKeyMin);
+
+ builder.append("\n");
+ builder.append("twUser:key:max");
+ builder.append(twUserKeyMax);
+
+ builder.append("\n");
+ builder.append("fbMessage:key:min");
+ builder.append(fbMessageIdMin);
+
+ builder.append("\n");
+ builder.append("fbMessage:key:max");
+ builder.append(fbMessageIdMax);
+
+ builder.append("\n");
+ builder.append("twMessage:key:min");
+ builder.append(twMessageIdMin);
+
+ builder.append("\n");
+ builder.append("twMessage:key:max");
+ builder.append(twMessageIdMax);
+
+ builder.append("\n");
+ builder.append("twMessage:key:max");
+ builder.append(twUserKeyMin);
+
+ return new String(builder);
+ }
+
+ @Override
+ public String getName() {
+ return "Partition";
+ }
+
+ }
+
+ interface ILibraryElement {
+
+ public enum ElementType {
+ PARTITION
+ }
+
+ public String getName();
+
+ }
+
+ public static class Configuration {
+
+ private final float numMB;
+ private final String unit;
+
+ private final List<SourcePartition> sourcePartitions;
+ private List<TargetPartition> targetPartitions;
+
+ public Configuration(float numMB, String unit, List<SourcePartition> partitions) throws IOException {
+ this.numMB = numMB;
+ this.unit = unit;
+ this.sourcePartitions = partitions;
+
+ }
+
+ public float getNumMB() {
+ return numMB;
+ }
+
+ public String getUnit() {
+ return unit;
+ }
+
+ public List<SourcePartition> getSourcePartitions() {
+ return sourcePartitions;
+ }
+
+ public List<TargetPartition> getTargetPartitions() {
+ return targetPartitions;
+ }
+
+ public void setTargetPartitions(List<TargetPartition> targetPartitions) {
+ this.targetPartitions = targetPartitions;
+ }
+
+ }
+
+ public static class XMLUtil {
+
+ public static void writeToXML(Configuration conf, String filePath) throws IOException,
+ ParserConfigurationException, TransformerException {
+
+ DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance();
+ DocumentBuilder docBuilder = docFactory.newDocumentBuilder();
+
+ // root elements
+ Document doc = docBuilder.newDocument();
+ Element rootElement = doc.createElement("Partitions");
+ doc.appendChild(rootElement);
+
+ int index = 0;
+ for (TargetPartition partition : conf.getTargetPartitions()) {
+ writePartitionElement(conf.getSourcePartitions().get(index), partition, rootElement, doc);
+ }
+
+ TransformerFactory transformerFactory = TransformerFactory.newInstance();
+ Transformer transformer = transformerFactory.newTransformer();
+
+ transformer.setOutputProperty(OutputKeys.ENCODING, "utf-8");
+ transformer.setOutputProperty(OutputKeys.OMIT_XML_DECLARATION, "no");
+ transformer.setOutputProperty(OutputKeys.INDENT, "yes");
+ transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "2");
+
+ DOMSource source = new DOMSource(doc);
+ StreamResult result = new StreamResult(new File(filePath));
+
+ transformer.transform(source, result);
+
+ }
+
+ public static void writePartitionInfo(Configuration conf, String filePath) throws IOException {
+ BufferedWriter bw = new BufferedWriter(new FileWriter(filePath));
+ for (SourcePartition sp : conf.getSourcePartitions()) {
+ bw.write(sp.getHost() + ":" + sp.getName() + ":" + sp.getPath());
+ bw.write("\n");
+ }
+ bw.close();
+ }
+
+ public static Document getDocument(String filePath) throws Exception {
+ File inputFile = new File(filePath);
+ DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+ DocumentBuilder db = dbf.newDocumentBuilder();
+ Document doc = db.parse(inputFile);
+ doc.getDocumentElement().normalize();
+ return doc;
+ }
+
+ public static Configuration getConfiguration(String filePath) throws Exception {
+ Configuration conf = getConfiguration(getDocument(filePath));
+ PartitionMetrics metrics = new PartitionMetrics(conf.getNumMB(), conf.getUnit(), conf.getSourcePartitions()
+ .size());
+ List<TargetPartition> targetPartitions = getTargetPartitions(metrics, conf.getSourcePartitions());
+ conf.setTargetPartitions(targetPartitions);
+ return conf;
+ }
+
+ public static Configuration getConfiguration(Document document) throws IOException {
+ Element rootEle = document.getDocumentElement();
+ NodeList nodeList = rootEle.getChildNodes();
+ float size = Float.parseFloat(getStringValue((Element) nodeList, "size"));
+ String unit = getStringValue((Element) nodeList, "unit");
+ List<SourcePartition> sourcePartitions = getSourcePartitions(document);
+ return new Configuration(size, unit, sourcePartitions);
+ }
+
+ public static List<SourcePartition> getSourcePartitions(Document document) {
+ Element rootEle = document.getDocumentElement();
+ NodeList nodeList = rootEle.getElementsByTagName("partition");
+ List<SourcePartition> sourcePartitions = new ArrayList<SourcePartition>();
+ for (int i = 0; i < nodeList.getLength(); i++) {
+ Node node = nodeList.item(i);
+ sourcePartitions.add(getSourcePartition((Element) node));
+ }
+ return sourcePartitions;
+ }
+
+ public static SourcePartition getSourcePartition(Element functionElement) {
+ String name = getStringValue(functionElement, "name");
+ String host = getStringValue(functionElement, "host");
+ String path = getStringValue(functionElement, "path");
+ SourcePartition sp = new SourcePartition(name, host, path);
+ return sp;
+ }
+
+ public static String getStringValue(Element element, String tagName) {
+ String textValue = null;
+ NodeList nl = element.getElementsByTagName(tagName);
+ if (nl != null && nl.getLength() > 0) {
+ Element el = (Element) nl.item(0);
+ textValue = el.getFirstChild().getNodeValue();
+ }
+ return textValue;
+ }
+
+ public static PartitionConfiguration getPartitionConfiguration(String filePath, String partitionName)
+ throws Exception {
+ PartitionConfiguration pconf = getPartitionConfigurations(getDocument(filePath), partitionName);
+ return pconf;
+ }
+
+ public static PartitionConfiguration getPartitionConfigurations(Document document, String partitionName)
+ throws IOException {
+
+ Element rootEle = document.getDocumentElement();
+ NodeList nodeList = rootEle.getElementsByTagName("Partition");
+
+ for (int i = 0; i < nodeList.getLength(); i++) {
+ Node node = nodeList.item(i);
+ Element nodeElement = (Element) node;
+ String name = getStringValue(nodeElement, "name");
+ if (!name.equalsIgnoreCase(partitionName)) {
+ continue;
+ }
+ String host = getStringValue(nodeElement, "host");
+ String path = getStringValue(nodeElement, "path");
+
+ String fbUserKeyMin = getStringValue(nodeElement, "fbUserKeyMin");
+ String fbUserKeyMax = getStringValue(nodeElement, "fbUserKeyMax");
+ String twUserKeyMin = getStringValue(nodeElement, "twUserKeyMin");
+ String twUserKeyMax = getStringValue(nodeElement, "twUserKeyMax");
+ String fbMessageKeyMin = getStringValue(nodeElement, "fbMessageKeyMin");
+
+ String fbMessageKeyMax = getStringValue(nodeElement, "fbMessageKeyMax");
+ String twMessageKeyMin = getStringValue(nodeElement, "twMessageKeyMin");
+ String twMessageKeyMax = getStringValue(nodeElement, "twMessageKeyMax");
+ String numCommonUsers = getStringValue(nodeElement, "numCommonUsers");
+
+ SourcePartition sp = new SourcePartition(name, host, path);
+
+ TargetPartition tp = new TargetPartition(partitionName, host, path, Integer.parseInt(fbUserKeyMin),
+ Integer.parseInt(fbUserKeyMax), Integer.parseInt(twUserKeyMin), Integer.parseInt(twUserKeyMax),
+ Integer.parseInt(fbMessageKeyMin), Integer.parseInt(fbMessageKeyMax),
+ Integer.parseInt(twMessageKeyMin), Integer.parseInt(twMessageKeyMax),
+ Integer.parseInt(numCommonUsers));
+ PartitionConfiguration pc = new PartitionConfiguration(sp, tp);
+ return pc;
+ }
+ return null;
+ }
+
+ public static List<TargetPartition> getTargetPartitions(PartitionMetrics metrics,
+ List<SourcePartition> sourcePartitions) {
+ List<TargetPartition> partitions = new ArrayList<TargetPartition>();
+ int fbUserKeyMin = 1;
+ int twUserKeyMin = 1;
+ int fbMessageIdMin = 1;
+ int twMessageIdMin = 1;
+
+ for (SourcePartition sp : sourcePartitions) {
+ int fbUserKeyMax = fbUserKeyMin + metrics.getFbOnlyUsers() + metrics.getCommonUsers() - 1;
+ int twUserKeyMax = twUserKeyMin + metrics.getTwitterOnlyUsers() + metrics.getCommonUsers() - 1;
+
+ int fbMessageIdMax = fbMessageIdMin + metrics.getFbMessages() - 1;
+ int twMessageIdMax = twMessageIdMin + metrics.getTwMessages() - 1;
+ TargetPartition pe = new TargetPartition(sp.getName(), sp.getHost(), sp.getPath(), fbUserKeyMin,
+ fbUserKeyMax, twUserKeyMin, twUserKeyMax, fbMessageIdMin, fbMessageIdMax, twMessageIdMin,
+ twMessageIdMax, metrics.getCommonUsers());
+ partitions.add(pe);
+
+ fbUserKeyMin = fbUserKeyMax + 1;
+ twUserKeyMin = twUserKeyMax + 1;
+
+ fbMessageIdMin = fbMessageIdMax + 1;
+ twMessageIdMin = twMessageIdMax + 1;
+ }
+
+ return partitions;
+ }
+
+ public static void writePartitionElement(SourcePartition sourcePartition, TargetPartition partition,
+ Element rootElement, Document doc) {
+ // staff elements
+ Element pe = doc.createElement("Partition");
+ rootElement.appendChild(pe);
+
+ // name element
+ Element name = doc.createElement("name");
+ name.appendChild(doc.createTextNode("" + partition.getName()));
+ pe.appendChild(name);
+
+ // host element
+ Element host = doc.createElement("host");
+ host.appendChild(doc.createTextNode("" + partition.getHost()));
+ pe.appendChild(host);
+
+ // path element
+ Element path = doc.createElement("path");
+ path.appendChild(doc.createTextNode("" + partition.getPath()));
+ pe.appendChild(path);
+
+ // fbUserKeyMin element
+ Element fbUserKeyMin = doc.createElement("fbUserKeyMin");
+ fbUserKeyMin.appendChild(doc.createTextNode("" + partition.getFbUserKeyMin()));
+ pe.appendChild(fbUserKeyMin);
+
+ // fbUserKeyMax element
+ Element fbUserKeyMax = doc.createElement("fbUserKeyMax");
+ fbUserKeyMax.appendChild(doc.createTextNode("" + partition.getFbUserKeyMax()));
+ pe.appendChild(fbUserKeyMax);
+
+ // twUserKeyMin element
+ Element twUserKeyMin = doc.createElement("twUserKeyMin");
+ twUserKeyMin.appendChild(doc.createTextNode("" + partition.getTwUserKeyMin()));
+ pe.appendChild(twUserKeyMin);
+
+ // twUserKeyMax element
+ Element twUserKeyMax = doc.createElement("twUserKeyMax");
+ twUserKeyMax.appendChild(doc.createTextNode("" + partition.getTwUserKeyMax()));
+ pe.appendChild(twUserKeyMax);
+
+ // fbMessgeKeyMin element
+ Element fbMessageKeyMin = doc.createElement("fbMessageKeyMin");
+ fbMessageKeyMin.appendChild(doc.createTextNode("" + partition.getFbMessageIdMin()));
+ pe.appendChild(fbMessageKeyMin);
+
+ // fbMessgeKeyMin element
+ Element fbMessageKeyMax = doc.createElement("fbMessageKeyMax");
+ fbMessageKeyMax.appendChild(doc.createTextNode("" + partition.getFbMessageIdMax()));
+ pe.appendChild(fbMessageKeyMax);
+
+ // twMessgeKeyMin element
+ Element twMessageKeyMin = doc.createElement("twMessageKeyMin");
+ twMessageKeyMin.appendChild(doc.createTextNode("" + partition.getTwMessageIdMin()));
+ pe.appendChild(twMessageKeyMin);
+
+ // twMessgeKeyMin element
+ Element twMessageKeyMax = doc.createElement("twMessageKeyMax");
+ twMessageKeyMax.appendChild(doc.createTextNode("" + partition.getTwMessageIdMax()));
+ pe.appendChild(twMessageKeyMax);
+
+ // twMessgeKeyMin element
+ Element numCommonUsers = doc.createElement("numCommonUsers");
+ numCommonUsers.appendChild(doc.createTextNode("" + partition.getCommonUsers()));
+ pe.appendChild(numCommonUsers);
+
+ }
+
+ public static void main(String args[]) throws Exception {
+ String confFile = "/Users/rgrove1/work/research/asterix/icde/data-gen/conf/conf.xml";
+ String outputPath = "/Users/rgrove1/work/research/asterix/icde/data-gen/output/conf-output.xml";
+ Configuration conf = getConfiguration(confFile);
+ writeToXML(conf, outputPath);
+ }
+
+ }
+
+ public static class Date {
+
+ private int day;
+ private int month;
+ private int year;
+
+ public Date(int month, int day, int year) {
+ this.month = month;
+ this.day = day;
+ this.year = year;
+ }
+
+ public void reset(int month, int day, int year) {
+ this.month = month;
+ this.day = day;
+ this.year = year;
+ }
+
+ public int getDay() {
+ return day;
+ }
+
+ public int getMonth() {
+ return month;
+ }
+
+ public int getYear() {
+ return year;
+ }
+
+ public Date() {
+ }
+
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("date");
+ builder.append("(\"");
+ builder.append(year);
+ builder.append("-");
+ builder.append(month < 10 ? "0" + month : "" + month);
+ builder.append("-");
+ builder.append(day < 10 ? "0" + day : "" + day);
+ builder.append("\")");
+ return builder.toString();
+ }
+
+ public void setDay(int day) {
+ this.day = day;
+ }
+
+ public void setMonth(int month) {
+ this.month = month;
+ }
+
+ public void setYear(int year) {
+ this.year = year;
+ }
+ }
+
+ public static class PartitionMetrics {
+
+ private final int fbMessages;
+ private final int twMessages;
+
+ private final int fbOnlyUsers;
+ private final int twitterOnlyUsers;
+ private final int commonUsers;
+
+ public PartitionMetrics(float number, String unit, int numPartitions) throws IOException {
+
+ int factor = 0;
+ if (unit.equalsIgnoreCase("MB")) {
+ factor = 1024 * 1024;
+ } else if (unit.equalsIgnoreCase("GB")) {
+ factor = 1024 * 1024 * 1024;
+ } else if (unit.equalsIgnoreCase("TB")) {
+ factor = 1024 * 1024 * 1024 * 1024;
+ } else
+ throw new IOException("Invalid unit:" + unit);
+
+ fbMessages = (int) (((number * factor * 0.80) / 258) / numPartitions);
+ twMessages = (int) (fbMessages * 1.1 / 0.35);
+
+ fbOnlyUsers = (int) ((number * factor * 0.20 * 0.0043)) / numPartitions;
+ twitterOnlyUsers = (int) (0.25 * fbOnlyUsers);
+ commonUsers = (int) (0.1 * fbOnlyUsers);
+ }
+
+ public int getFbMessages() {
+ return fbMessages;
+ }
+
+ public int getTwMessages() {
+ return twMessages;
+ }
+
+ public int getFbOnlyUsers() {
+ return fbOnlyUsers;
+ }
+
+ public int getTwitterOnlyUsers() {
+ return twitterOnlyUsers;
+ }
+
+ public int getCommonUsers() {
+ return commonUsers;
+ }
+
+ }
+
+ public static String[] lastNames = { "Hoopengarner", "Harrow", "Gardner", "Blyant", "Best", "Buttermore", "Gronko",
+ "Mayers", "Countryman", "Neely", "Ruhl", "Taggart", "Bash", "Cason", "Hil", "Zalack", "Mingle", "Carr",
+ "Rohtin", "Wardle", "Pullman", "Wire", "Kellogg", "Hiles", "Keppel", "Bratton", "Sutton", "Wickes",
+ "Muller", "Friedline", "Llora", "Elizabeth", "Anderson", "Gaskins", "Rifler", "Vinsant", "Stanfield",
+ "Black", "Guest", "Hujsak", "Carter", "Weidemann", "Hays", "Patton", "Hayhurst", "Paynter", "Cressman",
+ "Fiddler", "Evans", "Sherlock", "Woodworth", "Jackson", "Bloise", "Schneider", "Ring", "Kepplinger",
+ "James", "Moon", "Bennett", "Bashline", "Ryals", "Zeal", "Christman", "Milliron", "Nash", "Ewing", "Camp",
+ "Mason", "Richardson", "Bowchiew", "Hahn", "Wilson", "Wood", "Toyley", "Williamson", "Lafortune", "Errett",
+ "Saltser", "Hirleman", "Brindle", "Newbiggin", "Ulery", "Lambert", "Shick", "Kuster", "Moore", "Finck",
+ "Powell", "Jolce", "Townsend", "Sauter", "Cowher", "Wolfe", "Cavalet", "Porter", "Laborde", "Ballou",
+ "Murray", "Stoddard", "Pycroft", "Milne", "King", "Todd", "Staymates", "Hall", "Romanoff", "Keilbach",
+ "Sandford", "Hamilton", "Fye", "Kline", "Weeks", "Mcelroy", "Mccullough", "Bryant", "Hill", "Moore",
+ "Ledgerwood", "Prevatt", "Eckert", "Read", "Hastings", "Doverspike", "Allshouse", "Bryan", "Mccallum",
+ "Lombardi", "Mckendrick", "Cattley", "Barkley", "Steiner", "Finlay", "Priebe", "Armitage", "Hall", "Elder",
+ "Erskine", "Hatcher", "Walker", "Pearsall", "Dunkle", "Haile", "Adams", "Miller", "Newbern", "Basinger",
+ "Fuhrer", "Brinigh", "Mench", "Blackburn", "Bastion", "Mccune", "Bridger", "Hynes", "Quinn", "Courtney",
+ "Geddinge", "Field", "Seelig", "Cable", "Earhart", "Harshman", "Roby", "Beals", "Berry", "Reed", "Hector",
+ "Pittman", "Haverrman", "Kalp", "Briner", "Joghs", "Cowart", "Close", "Wynne", "Harden", "Weldy",
+ "Stephenson", "Hildyard", "Moberly", "Wells", "Mackendoerfer", "Fisher", "Oppie", "Oneal", "Churchill",
+ "Keister", "Alice", "Tavoularis", "Fisher", "Hair", "Burns", "Veith", "Wile", "Fuller", "Fields", "Clark",
+ "Randolph", "Stone", "Mcclymonds", "Holtzer", "Donkin", "Wilkinson", "Rosensteel", "Albright", "Stahl",
+ "Fox", "Kadel", "Houser", "Hanseu", "Henderson", "Davis", "Bicknell", "Swain", "Mercer", "Holdeman",
+ "Enderly", "Caesar", "Margaret", "Munshower", "Elless", "Lucy", "Feufer", "Schofield", "Graham",
+ "Blatenberger", "Benford", "Akers", "Campbell", "Ann", "Sadley", "Ling", "Gongaware", "Schmidt", "Endsley",
+ "Groah", "Flanders", "Reichard", "Lowstetter", "Sandblom", "Griffis", "Basmanoff", "Coveney", "Hawker",
+ "Archibald", "Hutton", "Barnes", "Diegel", "Raybould", "Focell", "Breitenstein", "Murray", "Chauvin",
+ "Busk", "Pheleps", "Teagarden", "Northey", "Baumgartner", "Fleming", "Harris", "Parkinson", "Carpenter",
+ "Whirlow", "Bonner", "Wortman", "Rogers", "Scott", "Lowe", "Mckee", "Huston", "Bullard", "Throckmorton",
+ "Rummel", "Mathews", "Dull", "Saline", "Tue", "Woolery", "Lalty", "Schrader", "Ramsey", "Eisenmann",
+ "Philbrick", "Sybilla", "Wallace", "Fonblanque", "Paul", "Orbell", "Higgens", "Casteel", "Franks",
+ "Demuth", "Eisenman", "Hay", "Robinson", "Fischer", "Hincken", "Wylie", "Leichter", "Bousum",
+ "Littlefield", "Mcdonald", "Greif", "Rhodes", "Wall", "Steele", "Baldwin", "Smith", "Stewart", "Schere",
+ "Mary", "Aultman", "Emrick", "Guess", "Mitchell", "Painter", "Aft", "Hasely", "Weldi", "Loewentsein",
+ "Poorbaugh", "Kepple", "Noton", "Judge", "Jackson", "Style", "Adcock", "Diller", "Marriman", "Johnston",
+ "Children", "Monahan", "Ehret", "Shaw", "Congdon", "Pinney", "Millard", "Crissman", "Tanner", "Rose",
+ "Knisely", "Cypret", "Sommer", "Poehl", "Hardie", "Bender", "Overholt", "Gottwine", "Beach", "Leslie",
+ "Trevithick", "Langston", "Magor", "Shotts", "Howe", "Hunter", "Cross", "Kistler", "Dealtry", "Christner",
+ "Pennington", "Thorley", "Eckhardstein", "Van", "Stroh", "Stough", "Stall", "Beedell", "Shea", "Garland",
+ "Mays", "Pritchard", "Frankenberger", "Rowley", "Lane", "Baum", "Alliman", "Park", "Jardine", "Butler",
+ "Cherry", "Kooser", "Baxter", "Billimek", "Downing", "Hurst", "Wood", "Baird", "Watkins", "Edwards",
+ "Kemerer", "Harding", "Owens", "Eiford", "Keener", "Garneis", "Fiscina", "Mang", "Draudy", "Mills",
+ "Gibson", "Reese", "Todd", "Ramos", "Levett", "Wilks", "Ward", "Mosser", "Dunlap", "Kifer", "Christopher",
+ "Ashbaugh", "Wynter", "Rawls", "Cribbs", "Haynes", "Thigpen", "Schreckengost", "Bishop", "Linton",
+ "Chapman", "James", "Jerome", "Hook", "Omara", "Houston", "Maclagan", "Sandys", "Pickering", "Blois",
+ "Dickson", "Kemble", "Duncan", "Woodward", "Southern", "Henley", "Treeby", "Cram", "Elsas", "Driggers",
+ "Warrick", "Overstreet", "Hindman", "Buck", "Sulyard", "Wentzel", "Swink", "Butt", "Schaeffer",
+ "Hoffhants", "Bould", "Willcox", "Lotherington", "Bagley", "Graff", "White", "Wheeler", "Sloan",
+ "Rodacker", "Hanford", "Jowers", "Kunkle", "Cass", "Powers", "Gilman", "Mcmichaels", "Hobbs", "Herndon",
+ "Prescott", "Smail", "Mcdonald", "Biery", "Orner", "Richards", "Mueller", "Isaman", "Bruxner", "Goodman",
+ "Barth", "Turzanski", "Vorrasi", "Stainforth", "Nehling", "Rahl", "Erschoff", "Greene", "Mckinnon",
+ "Reade", "Smith", "Pery", "Roose", "Greenwood", "Weisgarber", "Curry", "Holts", "Zadovsky", "Parrish",
+ "Putnam", "Munson", "Mcindoe", "Nickolson", "Brooks", "Bollinger", "Stroble", "Siegrist", "Fulton",
+ "Tomey", "Zoucks", "Roberts", "Otis", "Clarke", "Easter", "Johnson", "Fylbrigg", "Taylor", "Swartzbaugh",
+ "Weinstein", "Gadow", "Sayre", "Marcotte", "Wise", "Atweeke", "Mcfall", "Napier", "Eisenhart", "Canham",
+ "Sealis", "Baughman", "Gertraht", "Losey", "Laurence", "Eva", "Pershing", "Kern", "Pirl", "Rega",
+ "Sanborn", "Kanaga", "Sanders", "Anderson", "Dickinson", "Osteen", "Gettemy", "Crom", "Snyder", "Reed",
+ "Laurenzi", "Riggle", "Tillson", "Fowler", "Raub", "Jenner", "Koepple", "Soames", "Goldvogel", "Dimsdale",
+ "Zimmer", "Giesen", "Baker", "Beail", "Mortland", "Bard", "Sanner", "Knopsnider", "Jenkins", "Bailey",
+ "Werner", "Barrett", "Faust", "Agg", "Tomlinson", "Williams", "Little", "Greenawalt", "Wells", "Wilkins",
+ "Gisiko", "Bauerle", "Harrold", "Prechtl", "Polson", "Faast", "Winton", "Garneys", "Peters", "Potter",
+ "Porter", "Tennant", "Eve", "Dugger", "Jones", "Burch", "Cowper", "Whittier" };
+
+ public static String[] firstNames = { "Albert", "Jacquelin", "Dona", "Alia", "Mayme", "Genoveva", "Emma", "Lena",
+ "Melody", "Vilma", "Katelyn", "Jeremy", "Coral", "Leann", "Lita", "Gilda", "Kayla", "Alvina", "Maranda",
+ "Verlie", "Khadijah", "Karey", "Patrice", "Kallie", "Corey", "Mollie", "Daisy", "Melanie", "Sarita",
+ "Nichole", "Pricilla", "Terresa", "Berneice", "Arianne", "Brianne", "Lavinia", "Ulrike", "Lesha", "Adell",
+ "Ardelle", "Marisha", "Laquita", "Karyl", "Maryjane", "Kendall", "Isobel", "Raeann", "Heike", "Barbera",
+ "Norman", "Yasmine", "Nevada", "Mariam", "Edith", "Eugena", "Lovie", "Maren", "Bennie", "Lennie", "Tamera",
+ "Crystal", "Randi", "Anamaria", "Chantal", "Jesenia", "Avis", "Shela", "Randy", "Laurena", "Sharron",
+ "Christiane", "Lorie", "Mario", "Elizabeth", "Reina", "Adria", "Lakisha", "Brittni", "Azzie", "Dori",
+ "Shaneka", "Asuncion", "Katheryn", "Laurice", "Sharita", "Krystal", "Reva", "Inger", "Alpha", "Makeda",
+ "Anabel", "Loni", "Tiara", "Meda", "Latashia", "Leola", "Chin", "Daisey", "Ivory", "Amalia", "Logan",
+ "Tyler", "Kyong", "Carolann", "Maryetta", "Eufemia", "Anya", "Doreatha", "Lorna", "Rutha", "Ehtel",
+ "Debbie", "Chassidy", "Sang", "Christa", "Lottie", "Chun", "Karine", "Peggie", "Amina", "Melany", "Alayna",
+ "Scott", "Romana", "Naomi", "Christiana", "Salena", "Taunya", "Mitsue", "Regina", "Chelsie", "Charity",
+ "Dacia", "Aletha", "Latosha", "Lia", "Tamica", "Chery", "Bianca", "Shu", "Georgianne", "Myriam", "Austin",
+ "Wan", "Mallory", "Jana", "Georgie", "Jenell", "Kori", "Vicki", "Delfina", "June", "Mellisa", "Catherina",
+ "Claudie", "Tynisha", "Dayle", "Enriqueta", "Belen", "Pia", "Sarai", "Rosy", "Renay", "Kacie", "Frieda",
+ "Cayla", "Elissa", "Claribel", "Sabina", "Mackenzie", "Raina", "Cira", "Mitzie", "Aubrey", "Serafina",
+ "Maria", "Katharine", "Esperanza", "Sung", "Daria", "Billye", "Stefanie", "Kasha", "Holly", "Suzanne",
+ "Inga", "Flora", "Andria", "Genevie", "Eladia", "Janet", "Erline", "Renna", "Georgeanna", "Delorse",
+ "Elnora", "Rudy", "Rima", "Leanora", "Letisha", "Love", "Alverta", "Pinkie", "Domonique", "Jeannie",
+ "Jose", "Jacqueline", "Tara", "Lily", "Erna", "Tennille", "Galina", "Tamala", "Kirby", "Nichelle",
+ "Myesha", "Farah", "Santa", "Ludie", "Kenia", "Yee", "Micheline", "Maryann", "Elaina", "Ethelyn",
+ "Emmaline", "Shanell", "Marina", "Nila", "Alane", "Shakira", "Dorris", "Belinda", "Elois", "Barbie",
+ "Carita", "Gisela", "Lura", "Fransisca", "Helga", "Peg", "Leonarda", "Earlie", "Deetta", "Jacquetta",
+ "Blossom", "Kayleigh", "Deloras", "Keshia", "Christinia", "Dulce", "Bernie", "Sheba", "Lashanda", "Tula",
+ "Claretta", "Kary", "Jeanette", "Lupita", "Lenora", "Hisako", "Sherise", "Glynda", "Adela", "Chia",
+ "Sudie", "Mindy", "Caroyln", "Lindsey", "Xiomara", "Mercedes", "Onie", "Loan", "Alexis", "Tommie",
+ "Donette", "Monica", "Soo", "Camellia", "Lavera", "Valery", "Ariana", "Sophia", "Loris", "Ginette",
+ "Marielle", "Tari", "Julissa", "Alesia", "Suzanna", "Emelda", "Erin", "Ladawn", "Sherilyn", "Candice",
+ "Nereida", "Fairy", "Carl", "Joel", "Marilee", "Gracia", "Cordie", "So", "Shanita", "Drew", "Cassie",
+ "Sherie", "Marget", "Norma", "Delois", "Debera", "Chanelle", "Catarina", "Aracely", "Carlene", "Tricia",
+ "Aleen", "Katharina", "Marguerita", "Guadalupe", "Margorie", "Mandie", "Kathe", "Chong", "Sage", "Faith",
+ "Maryrose", "Stephany", "Ivy", "Pauline", "Susie", "Cristen", "Jenifer", "Annette", "Debi", "Karmen",
+ "Luci", "Shayla", "Hope", "Ocie", "Sharie", "Tami", "Breana", "Kerry", "Rubye", "Lashay", "Sondra",
+ "Katrice", "Brunilda", "Cortney", "Yan", "Zenobia", "Penni", "Addie", "Lavona", "Noel", "Anika",
+ "Herlinda", "Valencia", "Bunny", "Tory", "Victoria", "Carrie", "Mikaela", "Wilhelmina", "Chung",
+ "Hortencia", "Gerda", "Wen", "Ilana", "Sibyl", "Candida", "Victorina", "Chantell", "Casie", "Emeline",
+ "Dominica", "Cecila", "Delora", "Miesha", "Nova", "Sally", "Ronald", "Charlette", "Francisca", "Mina",
+ "Jenna", "Loraine", "Felisa", "Lulu", "Page", "Lyda", "Babara", "Flor", "Walter", "Chan", "Sherika",
+ "Kala", "Luna", "Vada", "Syreeta", "Slyvia", "Karin", "Renata", "Robbi", "Glenda", "Delsie", "Lizzie",
+ "Genia", "Caitlin", "Bebe", "Cory", "Sam", "Leslee", "Elva", "Caren", "Kasie", "Leticia", "Shannan",
+ "Vickey", "Sandie", "Kyle", "Chang", "Terrilyn", "Sandra", "Elida", "Marketta", "Elsy", "Tu", "Carman",
+ "Ashlie", "Vernia", "Albertine", "Vivian", "Elba", "Bong", "Margy", "Janetta", "Xiao", "Teofila", "Danyel",
+ "Nickole", "Aleisha", "Tera", "Cleotilde", "Dara", "Paulita", "Isela", "Maricela", "Rozella", "Marivel",
+ "Aurora", "Melissa", "Carylon", "Delinda", "Marvella", "Candelaria", "Deidre", "Tawanna", "Myrtie",
+ "Milagro", "Emilie", "Coretta", "Ivette", "Suzann", "Ammie", "Lucina", "Lory", "Tena", "Eleanor",
+ "Cherlyn", "Tiana", "Brianna", "Myra", "Flo", "Carisa", "Kandi", "Erlinda", "Jacqulyn", "Fermina", "Riva",
+ "Palmira", "Lindsay", "Annmarie", "Tamiko", "Carline", "Amelia", "Quiana", "Lashawna", "Veola", "Belva",
+ "Marsha", "Verlene", "Alex", "Leisha", "Camila", "Mirtha", "Melva", "Lina", "Arla", "Cythia", "Towanda",
+ "Aracelis", "Tasia", "Aurore", "Trinity", "Bernadine", "Farrah", "Deneen", "Ines", "Betty", "Lorretta",
+ "Dorethea", "Hertha", "Rochelle", "Juli", "Shenika", "Yung", "Lavon", "Deeanna", "Nakia", "Lynnette",
+ "Dinorah", "Nery", "Elene", "Carolee", "Mira", "Franchesca", "Lavonda", "Leida", "Paulette", "Dorine",
+ "Allegra", "Keva", "Jeffrey", "Bernardina", "Maryln", "Yoko", "Faviola", "Jayne", "Lucilla", "Charita",
+ "Ewa", "Ella", "Maggie", "Ivey", "Bettie", "Jerri", "Marni", "Bibi", "Sabrina", "Sarah", "Marleen",
+ "Katherin", "Remona", "Jamika", "Antonina", "Oliva", "Lajuana", "Fonda", "Sigrid", "Yael", "Billi",
+ "Verona", "Arminda", "Mirna", "Tesha", "Katheleen", "Bonita", "Kamilah", "Patrica", "Julio", "Shaina",
+ "Mellie", "Denyse", "Deandrea", "Alena", "Meg", "Kizzie", "Krissy", "Karly", "Alleen", "Yahaira", "Lucie",
+ "Karena", "Elaine", "Eloise", "Buena", "Marianela", "Renee", "Nan", "Carolynn", "Windy", "Avril", "Jane",
+ "Vida", "Thea", "Marvel", "Rosaline", "Tifany", "Robena", "Azucena", "Carlota", "Mindi", "Andera", "Jenny",
+ "Courtney", "Lyndsey", "Willette", "Kristie", "Shaniqua", "Tabatha", "Ngoc", "Una", "Marlena", "Louetta",
+ "Vernie", "Brandy", "Jacquelyne", "Jenelle", "Elna", "Erminia", "Ida", "Audie", "Louis", "Marisol",
+ "Shawana", "Harriette", "Karol", "Kitty", "Esmeralda", "Vivienne", "Eloisa", "Iris", "Jeanice", "Cammie",
+ "Jacinda", "Shena", "Floy", "Theda", "Lourdes", "Jayna", "Marg", "Kati", "Tanna", "Rosalyn", "Maxima",
+ "Soon", "Angelika", "Shonna", "Merle", "Kassandra", "Deedee", "Heidi", "Marti", "Renae", "Arleen",
+ "Alfredia", "Jewell", "Carley", "Pennie", "Corina", "Tonisha", "Natividad", "Lilliana", "Darcie", "Shawna",
+ "Angel", "Piedad", "Josefa", "Rebbeca", "Natacha", "Nenita", "Petrina", "Carmon", "Chasidy", "Temika",
+ "Dennise", "Renetta", "Augusta", "Shirlee", "Valeri", "Casimira", "Janay", "Berniece", "Deborah", "Yaeko",
+ "Mimi", "Digna", "Irish", "Cher", "Yong", "Lucila", "Jimmie", "Junko", "Lezlie", "Waneta", "Sandee",
+ "Marquita", "Eura", "Freeda", "Annabell", "Laree", "Jaye", "Wendy", "Toshia", "Kylee", "Aleta", "Emiko",
+ "Clorinda", "Sixta", "Audrea", "Juanita", "Birdie", "Reita", "Latanya", "Nia", "Leora", "Laurine",
+ "Krysten", "Jerrie", "Chantel", "Ira", "Sena", "Andre", "Jann", "Marla", "Precious", "Katy", "Gabrielle",
+ "Yvette", "Brook", "Shirlene", "Eldora", "Laura", "Milda", "Euna", "Jettie", "Debora", "Lise", "Edythe",
+ "Leandra", "Shandi", "Araceli", "Johanne", "Nieves", "Denese", "Carmelita", "Nohemi", "Annice", "Natalie",
+ "Yolande", "Jeffie", "Vashti", "Vickie", "Obdulia", "Youlanda", "Lupe", "Tomoko", "Monserrate", "Domitila",
+ "Etsuko", "Adrienne", "Lakesha", "Melissia", "Odessa", "Meagan", "Veronika", "Jolyn", "Isabelle", "Leah",
+ "Rhiannon", "Gianna", "Audra", "Sommer", "Renate", "Perla", "Thao", "Myong", "Lavette", "Mark", "Emilia",
+ "Ariane", "Karl", "Dorie", "Jacquie", "Mia", "Malka", "Shenita", "Tashina", "Christine", "Cherri", "Roni",
+ "Fran", "Mildred", "Sara", "Clarissa", "Fredia", "Elease", "Samuel", "Earlene", "Vernita", "Mae", "Concha",
+ "Renea", "Tamekia", "Hye", "Ingeborg", "Tessa", "Kelly", "Kristin", "Tam", "Sacha", "Kanisha", "Jillian",
+ "Tiffanie", "Ashlee", "Madelyn", "Donya", "Clementine", "Mickie", "My", "Zena", "Terrie", "Samatha",
+ "Gertie", "Tarra", "Natalia", "Sharlene", "Evie", "Shalon", "Rosalee", "Numbers", "Jodi", "Hattie",
+ "Naoma", "Valene", "Whitley", "Claude", "Alline", "Jeanne", "Camie", "Maragret", "Viola", "Kris", "Marlo",
+ "Arcelia", "Shari", "Jalisa", "Corrie", "Eleonor", "Angelyn", "Merry", "Lauren", "Melita", "Gita",
+ "Elenor", "Aurelia", "Janae", "Lyndia", "Margeret", "Shawanda", "Rolande", "Shirl", "Madeleine", "Celinda",
+ "Jaleesa", "Shemika", "Joye", "Tisa", "Trudie", "Kathrine", "Clarita", "Dinah", "Georgia", "Antoinette",
+ "Janis", "Suzette", "Sherri", "Herta", "Arie", "Hedy", "Cassi", "Audrie", "Caryl", "Jazmine", "Jessica",
+ "Beverly", "Elizbeth", "Marylee", "Londa", "Fredericka", "Argelia", "Nana", "Donnette", "Damaris",
+ "Hailey", "Jamee", "Kathlene", "Glayds", "Lydia", "Apryl", "Verla", "Adam", "Concepcion", "Zelda",
+ "Shonta", "Vernice", "Detra", "Meghann", "Sherley", "Sheri", "Kiyoko", "Margarita", "Adaline", "Mariela",
+ "Velda", "Ailene", "Juliane", "Aiko", "Edyth", "Cecelia", "Shavon", "Florance", "Madeline", "Rheba",
+ "Deann", "Ignacia", "Odelia", "Heide", "Mica", "Jennette", "Maricruz", "Ouida", "Darcy", "Laure",
+ "Justina", "Amada", "Laine", "Cruz", "Sunny", "Francene", "Roxanna", "Nam", "Nancie", "Deanna", "Letty",
+ "Britni", "Kazuko", "Lacresha", "Simon", "Caleb", "Milton", "Colton", "Travis", "Miles", "Jonathan",
+ "Logan", "Rolf", "Emilio", "Roberto", "Marcus", "Tim", "Delmar", "Devon", "Kurt", "Edward", "Jeffrey",
+ "Elvis", "Alfonso", "Blair", "Wm", "Sheldon", "Leonel", "Michal", "Federico", "Jacques", "Leslie",
+ "Augustine", "Hugh", "Brant", "Hong", "Sal", "Modesto", "Curtis", "Jefferey", "Adam", "John", "Glenn",
+ "Vance", "Alejandro", "Refugio", "Lucio", "Demarcus", "Chang", "Huey", "Neville", "Preston", "Bert",
+ "Abram", "Foster", "Jamison", "Kirby", "Erich", "Manual", "Dustin", "Derrick", "Donnie", "Jospeh", "Chris",
+ "Josue", "Stevie", "Russ", "Stanley", "Nicolas", "Samuel", "Waldo", "Jake", "Max", "Ernest", "Reinaldo",
+ "Rene", "Gale", "Morris", "Nathan", "Maximo", "Courtney", "Theodore", "Octavio", "Otha", "Delmer",
+ "Graham", "Dean", "Lowell", "Myles", "Colby", "Boyd", "Adolph", "Jarrod", "Nick", "Mark", "Clinton", "Kim",
+ "Sonny", "Dalton", "Tyler", "Jody", "Orville", "Luther", "Rubin", "Hollis", "Rashad", "Barton", "Vicente",
+ "Ted", "Rick", "Carmine", "Clifton", "Gayle", "Christopher", "Jessie", "Bradley", "Clay", "Theo", "Josh",
+ "Mitchell", "Boyce", "Chung", "Eugenio", "August", "Norbert", "Sammie", "Jerry", "Adan", "Edmundo",
+ "Homer", "Hilton", "Tod", "Kirk", "Emmett", "Milan", "Quincy", "Jewell", "Herb", "Steve", "Carmen",
+ "Bobby", "Odis", "Daron", "Jeremy", "Carl", "Hunter", "Tuan", "Thurman", "Asa", "Brenton", "Shane",
+ "Donny", "Andreas", "Teddy", "Dario", "Cyril", "Hoyt", "Teodoro", "Vincenzo", "Hilario", "Daren",
+ "Agustin", "Marquis", "Ezekiel", "Brendan", "Johnson", "Alden", "Richie", "Granville", "Chad", "Joseph",
+ "Lamont", "Jordon", "Gilberto", "Chong", "Rosendo", "Eddy", "Rob", "Dewitt", "Andre", "Titus", "Russell",
+ "Rigoberto", "Dick", "Garland", "Gabriel", "Hank", "Darius", "Ignacio", "Lazaro", "Johnie", "Mauro",
+ "Edmund", "Trent", "Harris", "Osvaldo", "Marvin", "Judson", "Rodney", "Randall", "Renato", "Richard",
+ "Denny", "Jon", "Doyle", "Cristopher", "Wilson", "Christian", "Jamie", "Roland", "Ken", "Tad", "Romeo",
+ "Seth", "Quinton", "Byron", "Ruben", "Darrel", "Deandre", "Broderick", "Harold", "Ty", "Monroe", "Landon",
+ "Mohammed", "Angel", "Arlen", "Elias", "Andres", "Carlton", "Numbers", "Tony", "Thaddeus", "Issac",
+ "Elmer", "Antoine", "Ned", "Fermin", "Grover", "Benito", "Abdul", "Cortez", "Eric", "Maxwell", "Coy",
+ "Gavin", "Rich", "Andy", "Del", "Giovanni", "Major", "Efren", "Horacio", "Joaquin", "Charles", "Noah",
+ "Deon", "Pasquale", "Reed", "Fausto", "Jermaine", "Irvin", "Ray", "Tobias", "Carter", "Yong", "Jorge",
+ "Brent", "Daniel", "Zane", "Walker", "Thad", "Shaun", "Jaime", "Mckinley", "Bradford", "Nathanial",
+ "Jerald", "Aubrey", "Virgil", "Abel", "Philip", "Chester", "Chadwick", "Dominick", "Britt", "Emmitt",
+ "Ferdinand", "Julian", "Reid", "Santos", "Dwain", "Morgan", "James", "Marion", "Micheal", "Eddie", "Brett",
+ "Stacy", "Kerry", "Dale", "Nicholas", "Darrick", "Freeman", "Scott", "Newton", "Sherman", "Felton",
+ "Cedrick", "Winfred", "Brad", "Fredric", "Dewayne", "Virgilio", "Reggie", "Edgar", "Heriberto", "Shad",
+ "Timmy", "Javier", "Nestor", "Royal", "Lynn", "Irwin", "Ismael", "Jonas", "Wiley", "Austin", "Kieth",
+ "Gonzalo", "Paris", "Earnest", "Arron", "Jarred", "Todd", "Erik", "Maria", "Chauncey", "Neil", "Conrad",
+ "Maurice", "Roosevelt", "Jacob", "Sydney", "Lee", "Basil", "Louis", "Rodolfo", "Rodger", "Roman", "Corey",
+ "Ambrose", "Cristobal", "Sylvester", "Benton", "Franklin", "Marcelo", "Guillermo", "Toby", "Jeramy",
+ "Donn", "Danny", "Dwight", "Clifford", "Valentine", "Matt", "Jules", "Kareem", "Ronny", "Lonny", "Son",
+ "Leopoldo", "Dannie", "Gregg", "Dillon", "Orlando", "Weston", "Kermit", "Damian", "Abraham", "Walton",
+ "Adrian", "Rudolf", "Will", "Les", "Norberto", "Fred", "Tyrone", "Ariel", "Terry", "Emmanuel", "Anderson",
+ "Elton", "Otis", "Derek", "Frankie", "Gino", "Lavern", "Jarod", "Kenny", "Dane", "Keenan", "Bryant",
+ "Eusebio", "Dorian", "Ali", "Lucas", "Wilford", "Jeremiah", "Warner", "Woodrow", "Galen", "Bob",
+ "Johnathon", "Amado", "Michel", "Harry", "Zachery", "Taylor", "Booker", "Hershel", "Mohammad", "Darrell",
+ "Kyle", "Stuart", "Marlin", "Hyman", "Jeffery", "Sidney", "Merrill", "Roy", "Garrett", "Porter", "Kenton",
+ "Giuseppe", "Terrance", "Trey", "Felix", "Buster", "Von", "Jackie", "Linwood", "Darron", "Francisco",
+ "Bernie", "Diego", "Brendon", "Cody", "Marco", "Ahmed", "Antonio", "Vince", "Brooks", "Kendrick", "Ross",
+ "Mohamed", "Jim", "Benny", "Gerald", "Pablo", "Charlie", "Antony", "Werner", "Hipolito", "Minh", "Mel",
+ "Derick", "Armand", "Fidel", "Lewis", "Donnell", "Desmond", "Vaughn", "Guadalupe", "Keneth", "Rodrick",
+ "Spencer", "Chas", "Gus", "Harlan", "Wes", "Carmelo", "Jefferson", "Gerard", "Jarvis", "Haywood", "Hayden",
+ "Sergio", "Gene", "Edgardo", "Colin", "Horace", "Dominic", "Aldo", "Adolfo", "Juan", "Man", "Lenard",
+ "Clement", "Everett", "Hal", "Bryon", "Mason", "Emerson", "Earle", "Laurence", "Columbus", "Lamar",
+ "Douglas", "Ian", "Fredrick", "Marc", "Loren", "Wallace", "Randell", "Noble", "Ricardo", "Rory", "Lindsey",
+ "Boris", "Bill", "Carlos", "Domingo", "Grant", "Craig", "Ezra", "Matthew", "Van", "Rudy", "Danial",
+ "Brock", "Maynard", "Vincent", "Cole", "Damion", "Ellsworth", "Marcel", "Markus", "Rueben", "Tanner",
+ "Reyes", "Hung", "Kennith", "Lindsay", "Howard", "Ralph", "Jed", "Monte", "Garfield", "Avery", "Bernardo",
+ "Malcolm", "Sterling", "Ezequiel", "Kristofer", "Luciano", "Casey", "Rosario", "Ellis", "Quintin",
+ "Trevor", "Miquel", "Jordan", "Arthur", "Carson", "Tyron", "Grady", "Walter", "Jonathon", "Ricky",
+ "Bennie", "Terrence", "Dion", "Dusty", "Roderick", "Isaac", "Rodrigo", "Harrison", "Zack", "Dee", "Devin",
+ "Rey", "Ulysses", "Clint", "Greg", "Dino", "Frances", "Wade", "Franklyn", "Jude", "Bradly", "Salvador",
+ "Rocky", "Weldon", "Lloyd", "Milford", "Clarence", "Alec", "Allan", "Bobbie", "Oswaldo", "Wilfred",
+ "Raleigh", "Shelby", "Willy", "Alphonso", "Arnoldo", "Robbie", "Truman", "Nicky", "Quinn", "Damien",
+ "Lacy", "Marcos", "Parker", "Burt", "Carroll", "Denver", "Buck", "Dong", "Normand", "Billie", "Edwin",
+ "Troy", "Arden", "Rusty", "Tommy", "Kenneth", "Leo", "Claud", "Joel", "Kendall", "Dante", "Milo", "Cruz",
+ "Lucien", "Ramon", "Jarrett", "Scottie", "Deshawn", "Ronnie", "Pete", "Alonzo", "Whitney", "Stefan",
+ "Sebastian", "Edmond", "Enrique", "Branden", "Leonard", "Loyd", "Olin", "Ron", "Rhett", "Frederic",
+ "Orval", "Tyrell", "Gail", "Eli", "Antonia", "Malcom", "Sandy", "Stacey", "Nickolas", "Hosea", "Santo",
+ "Oscar", "Fletcher", "Dave", "Patrick", "Dewey", "Bo", "Vito", "Blaine", "Randy", "Robin", "Winston",
+ "Sammy", "Edwardo", "Manuel", "Valentin", "Stanford", "Filiberto", "Buddy", "Zachariah", "Johnnie",
+ "Elbert", "Paul", "Isreal", "Jerrold", "Leif", "Owen", "Sung", "Junior", "Raphael", "Josef", "Donte",
+ "Allen", "Florencio", "Raymond", "Lauren", "Collin", "Eliseo", "Bruno", "Martin", "Lyndon", "Kurtis",
+ "Salvatore", "Erwin", "Michael", "Sean", "Davis", "Alberto", "King", "Rolland", "Joe", "Tory", "Chase",
+ "Dallas", "Vernon", "Beau", "Terrell", "Reynaldo", "Monty", "Jame", "Dirk", "Florentino", "Reuben", "Saul",
+ "Emory", "Esteban", "Michale", "Claudio", "Jacinto", "Kelley", "Levi", "Andrea", "Lanny", "Wendell",
+ "Elwood", "Joan", "Felipe", "Palmer", "Elmo", "Lawrence", "Hubert", "Rudolph", "Duane", "Cordell",
+ "Everette", "Mack", "Alan", "Efrain", "Trenton", "Bryan", "Tom", "Wilmer", "Clyde", "Chance", "Lou",
+ "Brain", "Justin", "Phil", "Jerrod", "George", "Kris", "Cyrus", "Emery", "Rickey", "Lincoln", "Renaldo",
+ "Mathew", "Luke", "Dwayne", "Alexis", "Jackson", "Gil", "Marty", "Burton", "Emil", "Glen", "Willian",
+ "Clemente", "Keven", "Barney", "Odell", "Reginald", "Aurelio", "Damon", "Ward", "Gustavo", "Harley",
+ "Peter", "Anibal", "Arlie", "Nigel", "Oren", "Zachary", "Scot", "Bud", "Wilbert", "Bart", "Josiah",
+ "Marlon", "Eldon", "Darryl", "Roger", "Anthony", "Omer", "Francis", "Patricia", "Moises", "Chuck",
+ "Waylon", "Hector", "Jamaal", "Cesar", "Julius", "Rex", "Norris", "Ollie", "Isaias", "Quentin", "Graig",
+ "Lyle", "Jeffry", "Karl", "Lester", "Danilo", "Mike", "Dylan", "Carlo", "Ryan", "Leon", "Percy", "Lucius",
+ "Jamel", "Lesley", "Joey", "Cornelius", "Rico", "Arnulfo", "Chet", "Margarito", "Ernie", "Nathanael",
+ "Amos", "Cleveland", "Luigi", "Alfonzo", "Phillip", "Clair", "Elroy", "Alva", "Hans", "Shon", "Gary",
+ "Jesus", "Cary", "Silas", "Keith", "Israel", "Willard", "Randolph", "Dan", "Adalberto", "Claude",
+ "Delbert", "Garry", "Mary", "Larry", "Riley", "Robt", "Darwin", "Barrett", "Steven", "Kelly", "Herschel",
+ "Darnell", "Scotty", "Armando", "Miguel", "Lawerence", "Wesley", "Garth", "Carol", "Micah", "Alvin",
+ "Billy", "Earl", "Pat", "Brady", "Cory", "Carey", "Bernard", "Jayson", "Nathaniel", "Gaylord", "Archie",
+ "Dorsey", "Erasmo", "Angelo", "Elisha", "Long", "Augustus", "Hobert", "Drew", "Stan", "Sherwood",
+ "Lorenzo", "Forrest", "Shawn", "Leigh", "Hiram", "Leonardo", "Gerry", "Myron", "Hugo", "Alvaro", "Leland",
+ "Genaro", "Jamey", "Stewart", "Elden", "Irving", "Olen", "Antone", "Freddy", "Lupe", "Joshua", "Gregory",
+ "Andrew", "Sang", "Wilbur", "Gerardo", "Merlin", "Williams", "Johnny", "Alex", "Tommie", "Jimmy",
+ "Donovan", "Dexter", "Gaston", "Tracy", "Jeff", "Stephen", "Berry", "Anton", "Darell", "Fritz", "Willis",
+ "Noel", "Mariano", "Crawford", "Zoey", "Alex", "Brianna", "Carlie", "Lloyd", "Cal", "Astor", "Randolf",
+ "Magdalene", "Trevelyan", "Terance", "Roy", "Kermit", "Harriett", "Crystal", "Laurinda", "Kiersten",
+ "Phyllida", "Liz", "Bettie", "Rena", "Colten", "Berenice", "Sindy", "Wilma", "Amos", "Candi", "Ritchie",
+ "Dirk", "Kathlyn", "Callista", "Anona", "Flossie", "Sterling", "Calista", "Regan", "Erica", "Jeana",
+ "Keaton", "York", "Nolan", "Daniel", "Benton", "Tommie", "Serenity", "Deanna", "Chas", "Heron", "Marlyn",
+ "Xylia", "Tristin", "Lyndon", "Andriana", "Madelaine", "Maddison", "Leila", "Chantelle", "Audrey",
+ "Connor", "Daley", "Tracee", "Tilda", "Eliot", "Merle", "Linwood", "Kathryn", "Silas", "Alvina",
+ "Phinehas", "Janis", "Alvena", "Zubin", "Gwendolen", "Caitlyn", "Bertram", "Hailee", "Idelle", "Homer",
+ "Jannah", "Delbert", "Rhianna", "Cy", "Jefferson", "Wayland", "Nona", "Tempest", "Reed", "Jenifer",
+ "Ellery", "Nicolina", "Aldous", "Prince", "Lexia", "Vinnie", "Doug", "Alberic", "Kayleen", "Woody",
+ "Rosanne", "Ysabel", "Skyler", "Twyla", "Geordie", "Leta", "Clive", "Aaron", "Scottie", "Celeste", "Chuck",
+ "Erle", "Lallie", "Jaycob", "Ray", "Carrie", "Laurita", "Noreen", "Meaghan", "Ulysses", "Andy", "Drogo",
+ "Dina", "Yasmin", "Mya", "Luvenia", "Urban", "Jacob", "Laetitia", "Sherry", "Love", "Michaela", "Deonne",
+ "Summer", "Brendon", "Sheena", "Mason", "Jayson", "Linden", "Salal", "Darrell", "Diana", "Hudson",
+ "Lennon", "Isador", "Charley", "April", "Ralph", "James", "Mina", "Jolyon", "Laurine", "Monna", "Carita",
+ "Munro", "Elsdon", "Everette", "Radclyffe", "Darrin", "Herbert", "Gawain", "Sheree", "Trudy", "Emmaline",
+ "Kassandra", "Rebecca", "Basil", "Jen", "Don", "Osborne", "Lilith", "Hannah", "Fox", "Rupert", "Paulene",
+ "Darius", "Wally", "Baptist", "Sapphire", "Tia", "Sondra", "Kylee", "Ashton", "Jepson", "Joetta", "Val",
+ "Adela", "Zacharias", "Zola", "Marmaduke", "Shannah", "Posie", "Oralie", "Brittany", "Ernesta", "Raymund",
+ "Denzil", "Daren", "Roosevelt", "Nelson", "Fortune", "Mariel", "Nick", "Jaden", "Upton", "Oz", "Margaux",
+ "Precious", "Albert", "Bridger", "Jimmy", "Nicola", "Rosalynne", "Keith", "Walt", "Della", "Joanna",
+ "Xenia", "Esmeralda", "Major", "Simon", "Rexana", "Stacy", "Calanthe", "Sherley", "Kaitlyn", "Graham",
+ "Ramsey", "Abbey", "Madlyn", "Kelvin", "Bill", "Rue", "Monica", "Caileigh", "Laraine", "Booker", "Jayna",
+ "Greta", "Jervis", "Sherman", "Kendrick", "Tommy", "Iris", "Geffrey", "Kaelea", "Kerr", "Garrick", "Jep",
+ "Audley", "Nic", "Bronte", "Beulah", "Patricia", "Jewell", "Deidra", "Cory", "Everett", "Harper",
+ "Charity", "Godfrey", "Jaime", "Sinclair", "Talbot", "Dayna", "Cooper", "Rosaline", "Jennie", "Eileen",
+ "Latanya", "Corinna", "Roxie", "Caesar", "Charles", "Pollie", "Lindsey", "Sorrel", "Dwight", "Jocelyn",
+ "Weston", "Shyla", "Valorie", "Bessie", "Josh", "Lessie", "Dayton", "Kathi", "Chasity", "Wilton", "Adam",
+ "William", "Ash", "Angela", "Ivor", "Ria", "Jazmine", "Hailey", "Jo", "Silvestra", "Ernie", "Clifford",
+ "Levi", "Matilda", "Quincey", "Camilla", "Delicia", "Phemie", "Laurena", "Bambi", "Lourdes", "Royston",
+ "Chastity", "Lynwood", "Elle", "Brenda", "Phoebe", "Timothy", "Raschelle", "Lilly", "Burt", "Rina",
+ "Rodney", "Maris", "Jaron", "Wilf", "Harlan", "Audra", "Vincent", "Elwyn", "Drew", "Wynter", "Ora",
+ "Lissa", "Virgil", "Xavier", "Chad", "Ollie", "Leyton", "Karolyn", "Skye", "Roni", "Gladys", "Dinah",
+ "Penny", "August", "Osmund", "Whitaker", "Brande", "Cornell", "Phil", "Zara", "Kilie", "Gavin", "Coty",
+ "Randy", "Teri", "Keira", "Pru", "Clemency", "Kelcey", "Nevil", "Poppy", "Gareth", "Christabel", "Bastian",
+ "Wynonna", "Roselyn", "Goddard", "Collin", "Trace", "Neal", "Effie", "Denys", "Virginia", "Richard",
+ "Isiah", "Harrietta", "Gaylord", "Diamond", "Trudi", "Elaine", "Jemmy", "Gage", "Annabel", "Quincy", "Syd",
+ "Marianna", "Philomena", "Aubree", "Kathie", "Jacki", "Kelley", "Bess", "Cecil", "Maryvonne", "Kassidy",
+ "Anselm", "Dona", "Darby", "Jamison", "Daryl", "Darell", "Teal", "Lennie", "Bartholomew", "Katie",
+ "Maybelline", "Kimball", "Elvis", "Les", "Flick", "Harley", "Beth", "Bidelia", "Montague", "Helen", "Ozzy",
+ "Stef", "Debra", "Maxene", "Stefanie", "Russ", "Avril", "Johnathan", "Orson", "Chelsey", "Josephine",
+ "Deshaun", "Wendell", "Lula", "Ferdinanda", "Greg", "Brad", "Kynaston", "Dena", "Russel", "Robertina",
+ "Misti", "Leon", "Anjelica", "Bryana", "Myles", "Judi", "Curtis", "Davin", "Kristia", "Chrysanta",
+ "Hayleigh", "Hector", "Osbert", "Eustace", "Cary", "Tansy", "Cayley", "Maryann", "Alissa", "Ike",
+ "Tranter", "Reina", "Alwilda", "Sidony", "Columbine", "Astra", "Jillie", "Stephania", "Jonah", "Kennedy",
+ "Ferdinand", "Allegria", "Donella", "Kelleigh", "Darian", "Eldreda", "Jayden", "Herbie", "Jake", "Winston",
+ "Vi", "Annie", "Cherice", "Hugo", "Tricia", "Haydee", "Cassarah", "Darden", "Mallory", "Alton", "Hadley",
+ "Romayne", "Lacey", "Ern", "Alayna", "Cecilia", "Seward", "Tilly", "Edgar", "Concordia", "Ibbie", "Dahlia",
+ "Oswin", "Stu", "Brett", "Maralyn", "Kristeen", "Dotty", "Robyn", "Nessa", "Tresha", "Guinevere",
+ "Emerson", "Haze", "Lyn", "Henderson", "Lexa", "Jaylen", "Gail", "Lizette", "Tiara", "Robbie", "Destiny",
+ "Alice", "Livia", "Rosy", "Leah", "Jan", "Zach", "Vita", "Gia", "Micheal", "Rowina", "Alysha", "Bobbi",
+ "Delores", "Osmond", "Karaugh", "Wilbur", "Kasandra", "Renae", "Kaety", "Dora", "Gaye", "Amaryllis",
+ "Katelyn", "Dacre", "Prudence", "Ebony", "Camron", "Jerrold", "Vivyan", "Randall", "Donna", "Misty",
+ "Damon", "Selby", "Esmund", "Rian", "Garry", "Julius", "Raelene", "Clement", "Dom", "Tibby", "Moss",
+ "Millicent", "Gwendoline", "Berry", "Ashleigh", "Lilac", "Quin", "Vere", "Creighton", "Harriet", "Malvina",
+ "Lianne", "Pearle", "Kizzie", "Kara", "Petula", "Jeanie", "Maria", "Pacey", "Victoria", "Huey", "Toni",
+ "Rose", "Wallis", "Diggory", "Josiah", "Delma", "Keysha", "Channing", "Prue", "Lee", "Ryan", "Sidney",
+ "Valerie", "Clancy", "Ezra", "Gilbert", "Clare", "Laz", "Crofton", "Mike", "Annabella", "Tara", "Eldred",
+ "Arthur", "Jaylon", "Peronel", "Paden", "Dot", "Marian", "Amyas", "Alexus", "Esmond", "Abbie", "Stanley",
+ "Brittani", "Vickie", "Errol", "Kimberlee", "Uland", "Ebenezer", "Howie", "Eveline", "Andrea", "Trish",
+ "Hopkin", "Bryanna", "Temperance", "Valarie", "Femie", "Alix", "Terrell", "Lewin", "Lorrin", "Happy",
+ "Micah", "Rachyl", "Sloan", "Gertrude", "Elizabeth", "Dorris", "Andra", "Bram", "Gary", "Jeannine",
+ "Maurene", "Irene", "Yolonda", "Jonty", "Coleen", "Cecelia", "Chantal", "Stuart", "Caris", "Ros",
+ "Kaleigh", "Mirabelle", "Kolby", "Primrose", "Susannah", "Ginny", "Jinny", "Dolly", "Lettice", "Sonny",
+ "Melva", "Ernest", "Garret", "Reagan", "Trenton", "Gallagher", "Edwin", "Nikolas", "Corrie", "Lynette",
+ "Ettie", "Sly", "Debbi", "Eudora", "Brittney", "Tacey", "Marius", "Anima", "Gordon", "Olivia", "Kortney",
+ "Shantel", "Kolleen", "Nevaeh", "Buck", "Sera", "Liliana", "Aric", "Kalyn", "Mick", "Libby", "Ingram",
+ "Alexandria", "Darleen", "Jacklyn", "Hughie", "Tyler", "Aida", "Ronda", "Deemer", "Taryn", "Laureen",
+ "Samantha", "Dave", "Hardy", "Baldric", "Montgomery", "Gus", "Ellis", "Titania", "Luke", "Chase", "Haidee",
+ "Mayra", "Isabell", "Trinity", "Milo", "Abigail", "Tacita", "Meg", "Hervey", "Natasha", "Sadie", "Holden",
+ "Dee", "Mansel", "Perry", "Randi", "Frederica", "Georgina", "Kolour", "Debbie", "Seraphina", "Elspet",
+ "Julyan", "Raven", "Zavia", "Jarvis", "Jaymes", "Grover", "Cairo", "Alea", "Jordon", "Braxton", "Donny",
+ "Rhoda", "Tonya", "Bee", "Alyssia", "Ashlyn", "Reanna", "Lonny", "Arlene", "Deb", "Jane", "Nikole",
+ "Bettina", "Harrison", "Tamzen", "Arielle", "Adelaide", "Faith", "Bridie", "Wilburn", "Fern", "Nan",
+ "Shaw", "Zeke", "Alan", "Dene", "Gina", "Alexa", "Bailey", "Sal", "Tammy", "Maximillian", "America",
+ "Sylvana", "Fitz", "Mo", "Marissa", "Cass", "Eldon", "Wilfrid", "Tel", "Joann", "Kendra", "Tolly",
+ "Leanne", "Ferdie", "Haven", "Lucas", "Marlee", "Cyrilla", "Red", "Phoenix", "Jazmin", "Carin", "Gena",
+ "Lashonda", "Tucker", "Genette", "Kizzy", "Winifred", "Melody", "Keely", "Kaylyn", "Radcliff", "Lettie",
+ "Foster", "Lyndsey", "Nicholas", "Farley", "Louisa", "Dana", "Dortha", "Francine", "Doran", "Bonita",
+ "Hal", "Sawyer", "Reginald", "Aislin", "Nathan", "Baylee", "Abilene", "Ladonna", "Maurine", "Shelly",
+ "Deandre", "Jasmin", "Roderic", "Tiffany", "Amanda", "Verity", "Wilford", "Gayelord", "Whitney", "Demelza",
+ "Kenton", "Alberta", "Kyra", "Tabitha", "Sampson", "Korey", "Lillian", "Edison", "Clayton", "Steph",
+ "Maya", "Dusty", "Jim", "Ronny", "Adrianne", "Bernard", "Harris", "Kiley", "Alexander", "Kisha", "Ethalyn",
+ "Patience", "Briony", "Indigo", "Aureole", "Makenzie", "Molly", "Sherilyn", "Barry", "Laverne", "Hunter",
+ "Rocky", "Tyreek", "Madalyn", "Phyliss", "Chet", "Beatrice", "Faye", "Lavina", "Madelyn", "Tracey",
+ "Gyles", "Patti", "Carlyn", "Stephanie", "Jackalyn", "Larrie", "Kimmy", "Isolda", "Emelina", "Lis",
+ "Zillah", "Cody", "Sheard", "Rufus", "Paget", "Mae", "Rexanne", "Luvinia", "Tamsen", "Rosanna", "Greig",
+ "Stacia", "Mabelle", "Quianna", "Lotus", "Delice", "Bradford", "Angus", "Cosmo", "Earlene", "Adrian",
+ "Arlie", "Noelle", "Sabella", "Isa", "Adelle", "Innocent", "Kirby", "Trixie", "Kenelm", "Nelda", "Melia",
+ "Kendal", "Dorinda", "Placid", "Linette", "Kam", "Sherisse", "Evan", "Ewart", "Janice", "Linton",
+ "Jacaline", "Charissa", "Douglas", "Aileen", "Kemp", "Oli", "Amethyst", "Rosie", "Nigella", "Sherill",
+ "Anderson", "Alanna", "Eric", "Claudia", "Jennifer", "Boniface", "Harriet", "Vernon", "Lucy", "Shawnee",
+ "Gerard", "Cecily", "Romey", "Randall", "Wade", "Lux", "Dawson", "Gregg", "Kade", "Roxanne", "Melinda",
+ "Rolland", "Rowanne", "Fannie", "Isidore", "Melia", "Harvie", "Salal", "Eleonor", "Jacquette", "Lavone",
+ "Shanika", "Tarquin", "Janet", "Josslyn", "Maegan", "Augusta", "Aubree", "Francene", "Martie", "Marisa",
+ "Tyreek", "Tatianna", "Caleb", "Sheridan", "Nellie", "Barbara", "Wat", "Jayla", "Esmaralda", "Graeme",
+ "Lavena", "Jemima", "Nikolas", "Triston", "Portia", "Kyla", "Marcus", "Raeburn", "Jamison", "Earl", "Wren",
+ "Leighton", "Lagina", "Lucasta", "Dina", "Amaranta", "Jessika", "Claud", "Bernard", "Winifred", "Ebba",
+ "Sammi", "Gall", "Chloe", "Ottoline", "Herbert", "Janice", "Gareth", "Channing", "Caleigh", "Kailee",
+ "Ralphie", "Tamzen", "Quincy", "Beaumont", "Albert", "Jadyn", "Violet", "Luanna", "Moriah", "Humbert",
+ "Jed", "Leona", "Hale", "Mitch", "Marlin", "Nivek", "Darwin", "Dirk", "Liliana", "Meadow", "Bernadine",
+ "Jorie", "Peyton", "Astra", "Roscoe", "Gina", "Lovell", "Jewel", "Romayne", "Rosy", "Imogene",
+ "Margaretta", "Lorinda", "Hopkin", "Bobby", "Flossie", "Bennie", "Horatio", "Jonah", "Lyn", "Deana",
+ "Juliana", "Blanch", "Wright", "Kendal", "Woodrow", "Tania", "Austyn", "Val", "Mona", "Charla", "Rudyard",
+ "Pamela", "Raven", "Zena", "Nicola", "Kaelea", "Conor", "Virgil", "Sonnie", "Goodwin", "Christianne",
+ "Linford", "Myron", "Denton", "Charita", "Brody", "Ginnie", "Harrison", "Jeanine", "Quin", "Isolda",
+ "Zoie", "Pearce", "Margie", "Larrie", "Angelina", "Marcia", "Jessamine", "Delilah", "Dick", "Luana",
+ "Delicia", "Lake", "Luvenia", "Vaughan", "Concordia", "Gayelord", "Cheyenne", "Felix", "Dorris", "Pen",
+ "Kristeen", "Parris", "Everitt", "Josephina", "Amy", "Tommie", "Adrian", "April", "Rosaline", "Zachery",
+ "Trace", "Phoebe", "Jenelle", "Kameron", "Katharine", "Media", "Colton", "Tad", "Quianna", "Kerenza",
+ "Greta", "Luvinia", "Pete", "Tonya", "Beckah", "Barbra", "Jon", "Tetty", "Corey", "Sylvana", "Kizzy",
+ "Korey", "Trey", "Haydee", "Penny", "Mandy", "Panda", "Coline", "Ramsey", "Sukie", "Annabel", "Sarina",
+ "Corbin", "Suzanna", "Rob", "Duana", "Shell", "Jason", "Eddy", "Rube", "Roseann", "Celia", "Brianne",
+ "Nerissa", "Jera", "Humphry", "Ashlynn", "Terrence", "Philippina", "Coreen", "Kolour", "Indiana", "Paget",
+ "Marlyn", "Hester", "Isbel", "Ocean", "Harris", "Leslie", "Vere", "Monroe", "Isabelle", "Bertie", "Clitus",
+ "Dave", "Alethea", "Lessie", "Louiza", "Madlyn", "Garland", "Wolf", "Lalo", "Donny", "Amabel", "Tianna",
+ "Louie", "Susie", "Mackenzie", "Renie", "Tess", "Marmaduke", "Gwendolen", "Bettina", "Beatrix", "Esmund",
+ "Minnie", "Carlie", "Barnabas", "Ruthie", "Honour", "Haylie", "Xavior", "Freddie", "Ericka", "Aretha",
+ "Edie", "Madelina", "Anson", "Tabby", "Derrick", "Jocosa", "Deirdre", "Aislin", "Chastity", "Abigail",
+ "Wynonna", "Zo", "Eldon", "Krystine", "Ghislaine", "Zavia", "Nolene", "Marigold", "Kelley", "Sylvester",
+ "Odell", "George", "Laurene", "Franklyn", "Clarice", "Mo", "Dustin", "Debbi", "Lina", "Tony", "Acacia",
+ "Hettie", "Natalee", "Marcie", "Brittany", "Elnora", "Rachel", "Dawn", "Basil", "Christal", "Anjelica",
+ "Fran", "Tawny", "Delroy", "Tameka", "Lillie", "Ceara", "Deanna", "Deshaun", "Ken", "Bradford", "Justina",
+ "Merle", "Draven", "Gretta", "Harriette", "Webster", "Nathaniel", "Anemone", "Coleen", "Ruth", "Chryssa",
+ "Hortensia", "Saffie", "Deonne", "Leopold", "Harlan", "Lea", "Eppie", "Lucinda", "Tilda", "Fanny", "Titty",
+ "Lockie", "Jepson", "Sherisse", "Maralyn", "Ethel", "Sly", "Ebenezer", "Canute", "Ella", "Freeman",
+ "Reuben", "Olivette", "Nona", "Rik", "Amice", "Kristine", "Kathie", "Jayne", "Jeri", "Mckenna", "Bertram",
+ "Kaylee", "Livia", "Gil", "Wallace", "Maryann", "Keeleigh", "Laurinda", "Doran", "Khloe", "Dakota",
+ "Yaron", "Kimberleigh", "Gytha", "Doris", "Marylyn", "Benton", "Linnette", "Esther", "Jakki", "Rowina",
+ "Marian", "Roselyn", "Norbert", "Maggie", "Caesar", "Phinehas", "Jerry", "Jasmine", "Antonette", "Miriam",
+ "Monna", "Maryvonne", "Jacquetta", "Bernetta", "Napier", "Annie", "Gladwin", "Sheldon", "Aric", "Elouise",
+ "Gawain", "Kristia", "Gabe", "Kyra", "Red", "Tod", "Dudley", "Lorraine", "Ryley", "Sabina", "Poppy",
+ "Leland", "Aileen", "Eglantine", "Alicia", "Jeni", "Addy", "Tiffany", "Geffrey", "Lavina", "Collin",
+ "Clover", "Vin", "Jerome", "Doug", "Vincent", "Florence", "Scarlet", "Celeste", "Desdemona", "Tiphanie",
+ "Kassandra", "Ashton", "Madison", "Art", "Magdalene", "Iona", "Josepha", "Anise", "Ferne", "Derek",
+ "Huffie", "Qiana", "Ysabel", "Tami", "Shannah", "Xavier", "Willard", "Winthrop", "Vickie", "Maura",
+ "Placid", "Tiara", "Reggie", "Elissa", "Isa", "Chrysanta", "Jeff", "Bessie", "Terri", "Amilia", "Brett",
+ "Daniella", "Damion", "Carolina", "Maximillian", "Travers", "Benjamin", "Oprah", "Darcy", "Yolanda",
+ "Nicolina", "Crofton", "Jarrett", "Kaitlin", "Shauna", "Keren", "Bevis", "Kalysta", "Sharron", "Alyssa",
+ "Blythe", "Zelma", "Caelie", "Norwood", "Billie", "Patrick", "Gary", "Cambria", "Tylar", "Mason", "Helen",
+ "Melyssa", "Gene", "Gilberta", "Carter", "Herbie", "Harmonie", "Leola", "Eugenia", "Clint", "Pauletta",
+ "Edwyna", "Georgina", "Teal", "Harper", "Izzy", "Dillon", "Kezia", "Evangeline", "Colene", "Madelaine",
+ "Zilla", "Rudy", "Dottie", "Caris", "Morton", "Marge", "Tacey", "Parker", "Troy", "Liza", "Lewin",
+ "Tracie", "Justine", "Dallas", "Linden", "Ray", "Loretta", "Teri", "Elvis", "Diane", "Julianna", "Manfred",
+ "Denise", "Eireen", "Ann", "Kenith", "Linwood", "Kathlyn", "Bernice", "Shelley", "Oswald", "Amedeus",
+ "Homer", "Tanzi", "Ted", "Ralphina", "Hyacinth", "Lotus", "Matthias", "Arlette", "Clark", "Cecil",
+ "Elspeth", "Alvena", "Noah", "Millard", "Brenden", "Cole", "Philipa", "Nina", "Thelma", "Iantha", "Reid",
+ "Jefferson", "Meg", "Elsie", "Shirlee", "Nathan", "Nancy", "Simona", "Racheal", "Carin", "Emory", "Delice",
+ "Kristi", "Karaugh", "Kaety", "Tilly", "Em", "Alanis", "Darrin", "Jerrie", "Hollis", "Cary", "Marly",
+ "Carita", "Jody", "Farley", "Hervey", "Rosalin", "Cuthbert", "Stewart", "Jodene", "Caileigh", "Briscoe",
+ "Dolores", "Sheree", "Eustace", "Nigel", "Detta", "Barret", "Rowland", "Kenny", "Githa", "Zoey", "Adela",
+ "Petronella", "Opal", "Coleman", "Niles", "Cyril", "Dona", "Alberic", "Allannah", "Jules", "Avalon",
+ "Hadley", "Thomas", "Renita", "Calanthe", "Heron", "Shawnda", "Chet", "Malina", "Manny", "Rina", "Frieda",
+ "Eveleen", "Deshawn", "Amos", "Raelene", "Paige", "Molly", "Nannie", "Ileen", "Brendon", "Milford",
+ "Unice", "Rebeccah", "Caedmon", "Gae", "Doreen", "Vivian", "Louis", "Raphael", "Vergil", "Lise", "Glenn",
+ "Karyn", "Terance", "Reina", "Jake", "Gordon", "Wisdom", "Isiah", "Gervase", "Fern", "Marylou", "Roddy",
+ "Justy", "Derick", "Shantelle", "Adam", "Chantel", "Madoline", "Emmerson", "Lexie", "Mickey", "Stephen",
+ "Dane", "Stacee", "Elwin", "Tracey", "Alexandra", "Ricky", "Ian", "Kasey", "Rita", "Alanna", "Georgene",
+ "Deon", "Zavier", "Ophelia", "Deforest", "Lowell", "Zubin", "Hardy", "Osmund", "Tabatha", "Debby",
+ "Katlyn", "Tallulah", "Priscilla", "Braden", "Wil", "Keziah", "Jen", "Aggie", "Korbin", "Lemoine",
+ "Barnaby", "Tranter", "Goldie", "Roderick", "Trina", "Emery", "Pris", "Sidony", "Adelle", "Tate", "Wilf",
+ "Zola", "Brande", "Chris", "Calanthia", "Lilly", "Kaycee", "Lashonda", "Jasmin", "Elijah", "Shantel",
+ "Simon", "Rosalind", "Jarod", "Kaylie", "Corrine", "Joselyn", "Archibald", "Mariabella", "Winton",
+ "Merlin", "Chad", "Ursula", "Kristopher", "Hewie", "Adrianna", "Lyndsay", "Jasmyn", "Tim", "Evette",
+ "Margaret", "Samson", "Bronte", "Terence", "Leila", "Candice", "Tori", "Jamey", "Coriander", "Conrad",
+ "Floyd", "Karen", "Lorin", "Maximilian", "Cairo", "Emily", "Yasmin", "Karolyn", "Bryan", "Lanny",
+ "Kimberly", "Rick", "Chaz", "Krystle", "Lyric", "Laura", "Garrick", "Flip", "Monty", "Brendan",
+ "Ermintrude", "Rayner", "Merla", "Titus", "Marva", "Patricia", "Leone", "Tracy", "Jaqueline", "Hallam",
+ "Delores", "Cressida", "Carlyle", "Leann", "Kelcey", "Laurence", "Ryan", "Reynold", "Mark", "Collyn",
+ "Audie", "Sammy", "Ellery", "Sallie", "Pamelia", "Adolph", "Lydia", "Titania", "Ron", "Bridger", "Aline",
+ "Read", "Kelleigh", "Weldon", "Irving", "Garey", "Diggory", "Evander", "Kylee", "Deidre", "Ormond",
+ "Laurine", "Reannon", "Arline", "Pat"
+
+ };
+
+ public static String[] jargon = { "wireless", "signal", "network", "3G", "plan", "touch-screen",
+ "customer-service", "reachability", "voice-command", "shortcut-menu", "customization", "platform", "speed",
+ "voice-clarity", "voicemail-service" };
+
+ public static String[] vendors = { "at&t", "verizon", "t-mobile", "sprint", "motorola", "samsung", "iphone" };
+
+ public static String[] org_list = { "Latsonity", "ganjalax", "Zuncan", "Lexitechno", "Hot-tech", "subtam",
+ "Coneflex", "Ganjatax", "physcane", "Tranzap", "Qvohouse", "Zununoing", "jaydax", "Keytech", "goldendexon",
+ "Villa-tech", "Trustbam", "Newcom", "Voltlane", "Ontohothex", "Ranhotfan", "Alphadax", "Transhigh",
+ "kin-ron", "Doublezone", "Solophase", "Vivaace", "silfind", "Basecone", "sonstreet", "Freshfix",
+ "Techitechi", "Kanelectrics", "linedexon", "Goldcity", "Newfase", "Technohow", "Zimcone", "Salthex",
+ "U-ron", "Solfix", "whitestreet", "Xx-technology", "Hexviafind", "over-it", "Strongtone", "Tripplelane",
+ "geomedia", "Scotcity", "Inchex", "Vaiatech", "Striptaxon", "Hatcom", "tresline", "Sanjodax", "freshdox",
+ "Sumlane", "Quadlane", "Newphase", "overtech", "Voltbam", "Icerunin", "Fixdintex", "Hexsanhex", "Statcode",
+ "Greencare", "U-electrics", "Zamcorporation", "Ontotanin", "Tanzimcare", "Groovetex", "Ganjastrip",
+ "Redelectronics", "Dandamace", "Whitemedia", "strongex", "Streettax", "highfax", "Mathtech", "Xx-drill",
+ "Sublamdox", "Unijobam", "Rungozoom", "Fixelectrics", "Villa-dox", "Ransaofan", "Plexlane", "itlab",
+ "Lexicone", "Fax-fax", "Viatechi", "Inchdox", "Kongreen", "Doncare", "Y-geohex", "Opeelectronics",
+ "Medflex", "Dancode", "Roundhex", "Labzatron", "Newhotplus", "Sancone", "Ronholdings", "Quoline",
+ "zoomplus", "Fix-touch", "Codetechno", "Tanzumbam", "Indiex", "Canline" };
+}
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGeneratorDriver.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGeneratorDriver.java
new file mode 100644
index 0000000..b2ba88e
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/DataGeneratorDriver.java
@@ -0,0 +1,18 @@
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.util.Iterator;
+
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.InitializationInfo;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.TweetMessage;
+
+public class DataGeneratorDriver {
+
+ public static void main(String[] args) {
+
+ DataGenerator.initialize(new InitializationInfo());
+ Iterator<TweetMessage> tweetIterator = DataGenerator.getTwitterMessageIterator();
+ while (tweetIterator.hasNext()) {
+ System.out.println(tweetIterator.next().toString());
+ }
+ }
+}
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GenericSocketFeedAdapter.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GenericSocketFeedAdapter.java
new file mode 100644
index 0000000..a536076
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GenericSocketFeedAdapter.java
@@ -0,0 +1,92 @@
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.external.dataset.adapter.StreamBasedAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+public class GenericSocketFeedAdapter extends StreamBasedAdapter implements IFeedAdapter {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final String KEY_PORT = "port";
+
+ private static final Logger LOGGER = Logger.getLogger(GenericSocketFeedAdapter.class.getName());
+
+ private Map<String, String> configuration;
+
+ private SocketFeedServer socketFeedServer;
+
+ private static final int DEFAULT_PORT = 2909;
+
+ public GenericSocketFeedAdapter(Map<String, String> configuration, ITupleParserFactory parserFactory,
+ ARecordType outputtype, IHyracksTaskContext ctx) throws AsterixException, IOException {
+ super(parserFactory, outputtype, ctx);
+ this.configuration = configuration;
+ String portValue = (String) this.configuration.get(KEY_PORT);
+ int port = portValue != null ? Integer.parseInt(portValue) : DEFAULT_PORT;
+ this.socketFeedServer = new SocketFeedServer(configuration, outputtype, port);
+ }
+
+ @Override
+ public void start(int partition, IFrameWriter writer) throws Exception {
+ super.start(partition, writer);
+ }
+
+ @Override
+ public InputStream getInputStream(int partition) throws IOException {
+ return socketFeedServer.getInputStream();
+ }
+
+ private static class SocketFeedServer {
+ private ServerSocket serverSocket;
+ private InputStream inputStream;
+
+ public SocketFeedServer(Map<String, String> configuration, ARecordType outputtype, int port)
+ throws IOException, AsterixException {
+ try {
+ serverSocket = new ServerSocket(port);
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("port: " + port + " unusable ");
+ }
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Feed server configured to use port: " + port);
+ }
+ }
+
+ public InputStream getInputStream() {
+ Socket socket;
+ try {
+ socket = serverSocket.accept();
+ inputStream = socket.getInputStream();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ return inputStream;
+ }
+
+ public void stop() throws IOException {
+ serverSocket.close();
+ }
+
+ }
+
+ @Override
+ public void stop() throws Exception {
+ socketFeedServer.stop();
+ }
+
+}
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GenericSocketFeedAdapterFactory.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GenericSocketFeedAdapterFactory.java
new file mode 100644
index 0000000..788bb4f
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GenericSocketFeedAdapterFactory.java
@@ -0,0 +1,74 @@
+/*
+x * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.external.adapter.factory.StreamBasedAdapterFactory;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * Factory class for creating @see{GenericSocketFeedAdapter} The
+ * adapter listens at a port for receiving data (from external world).
+ * Data received is transformed into Asterix Data Format (ADM) and stored into
+ * a dataset a configured in the Adapter configuration.
+ */
+public class GenericSocketFeedAdapterFactory extends StreamBasedAdapterFactory implements IGenericAdapterFactory {
+
+ /**
+ *
+ */
+ private static final long serialVersionUID = 1L;
+
+ private ARecordType outputType;
+
+ @Override
+ public String getName() {
+ return "generic_socket_feed";
+ }
+
+ @Override
+ public AdapterType getAdapterType() {
+ return AdapterType.GENERIC;
+ }
+
+ @Override
+ public SupportedOperation getSupportedOperations() {
+ return SupportedOperation.READ;
+ }
+
+ @Override
+ public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+ this.configuration = configuration;
+ outputType = (ARecordType) outputType;
+ this.configureFormat(outputType);
+ }
+
+ @Override
+ public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+ return new AlgebricksCountPartitionConstraint(1);
+ }
+
+ @Override
+ public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
+ return new GenericSocketFeedAdapter(configuration, parserFactory, outputType, ctx);
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
index e037ec6..a92cb55 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
@@ -18,43 +18,28 @@
import java.io.InputStream;
import java.util.Map;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.external.dataset.adapter.FileSystemBasedAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.ITypedDatasourceAdapter;
-import edu.uci.ics.asterix.feed.managed.adapter.IManagedFeedAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter;
import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.runtime.operators.file.ADMDataParser;
-import edu.uci.ics.asterix.runtime.operators.file.AbstractTupleParser;
-import edu.uci.ics.asterix.runtime.operators.file.DelimitedDataParser;
-import edu.uci.ics.asterix.runtime.operators.file.IDataParser;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
/**
- * An adapter that simulates a feed from the contents of a source file. The file can be on the local file
- * system or on HDFS. The feed ends when the content of the source file has been ingested.
+ * An adapter that simulates a feed from the contents of a source file. The file
+ * can be on the local file system or on HDFS. The feed ends when the content of
+ * the source file has been ingested.
*/
-public class RateControlledFileSystemBasedAdapter extends FileSystemBasedAdapter implements ITypedDatasourceAdapter,
- IManagedFeedAdapter {
+
+public class RateControlledFileSystemBasedAdapter extends FileSystemBasedAdapter implements IFeedAdapter {
private static final long serialVersionUID = 1L;
private FileSystemBasedAdapter coreAdapter;
- private String format;
- public RateControlledFileSystemBasedAdapter(ARecordType atype, Map<String, Object> configuration,
- FileSystemBasedAdapter coreAdapter, String format) throws Exception {
- super(atype);
- this.configuration = configuration;
+ public RateControlledFileSystemBasedAdapter(ARecordType atype, Map<String, String> configuration,
+ FileSystemBasedAdapter coreAdapter, String format, ITupleParserFactory parserFactory,
+ IHyracksTaskContext ctx) throws Exception {
+ super(parserFactory, atype, ctx);
this.coreAdapter = coreAdapter;
- this.format = format;
}
@Override
@@ -63,184 +48,8 @@
}
@Override
- public void initialize(IHyracksTaskContext ctx) throws Exception {
- coreAdapter.initialize(ctx);
- this.ctx = ctx;
- }
-
- @Override
- public void configure(Map<String, Object> arguments) throws Exception {
- coreAdapter.configure(arguments);
- }
-
- @Override
- public AdapterType getAdapterType() {
- return coreAdapter.getAdapterType();
- }
-
- @Override
- protected ITupleParser getTupleParser() throws Exception {
- ITupleParser parser = null;
- if (format.equals(FORMAT_DELIMITED_TEXT)) {
- parser = getRateControlledDelimitedDataTupleParser((ARecordType) atype);
- } else if (format.equals(FORMAT_ADM)) {
- parser = getRateControlledADMDataTupleParser((ARecordType) atype);
- } else {
- throw new IllegalArgumentException(" format " + configuration.get(KEY_FORMAT) + " not supported");
- }
- return parser;
-
- }
-
- protected ITupleParser getRateControlledDelimitedDataTupleParser(ARecordType recordType) throws AsterixException,
- HyracksDataException {
- ITupleParser parser;
- int n = recordType.getFieldTypes().length;
- IValueParserFactory[] fieldParserFactories = new IValueParserFactory[n];
- for (int i = 0; i < n; i++) {
- ATypeTag tag = recordType.getFieldTypes()[i].getTypeTag();
- IValueParserFactory vpf = typeToValueParserFactMap.get(tag);
- if (vpf == null) {
- throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
- }
- fieldParserFactories[i] = vpf;
-
- }
- String delimiterValue = (String) configuration.get(KEY_DELIMITER);
- if (delimiterValue != null && delimiterValue.length() > 1) {
- throw new AsterixException("improper delimiter");
- }
-
- Character delimiter = delimiterValue.charAt(0);
- parser = new RateControlledTupleParserFactory(recordType, fieldParserFactories, delimiter, configuration)
- .createTupleParser(ctx);
- return parser;
- }
-
- protected ITupleParser getRateControlledADMDataTupleParser(ARecordType recordType) throws AsterixException {
- ITupleParser parser = null;
- try {
- parser = new RateControlledTupleParserFactory(recordType, configuration).createTupleParser(ctx);
- return parser;
- } catch (Exception e) {
- throw new AsterixException(e);
- }
-
- }
-
- @Override
- public ARecordType getAdapterOutputType() {
- return (ARecordType) atype;
- }
-
- @Override
- public void alter(Map<String, String> properties) {
- ((RateControlledTupleParser) parser).setInterTupleInterval(Long.parseLong(properties
- .get(RateControlledTupleParser.INTER_TUPLE_INTERVAL)));
- }
-
- @Override
public void stop() {
- ((RateControlledTupleParser) parser).stop();
+ ((RateControlledTupleParser) tupleParser).stop();
}
- @Override
- public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
- return coreAdapter.getPartitionConstraint();
- }
-}
-
-class RateControlledTupleParserFactory implements ITupleParserFactory {
-
- private static final long serialVersionUID = 1L;
-
- private final ARecordType recordType;
- private final IDataParser dataParser;
- private final Map<String, Object> configuration;
-
- public RateControlledTupleParserFactory(ARecordType recordType, IValueParserFactory[] valueParserFactories,
- char fieldDelimiter, Map<String, Object> configuration) {
- this.recordType = recordType;
- dataParser = new DelimitedDataParser(recordType, valueParserFactories, fieldDelimiter);
- this.configuration = configuration;
- }
-
- public RateControlledTupleParserFactory(ARecordType recordType, Map<String, Object> configuration) {
- this.recordType = recordType;
- dataParser = new ADMDataParser();
- this.configuration = configuration;
- }
-
- @Override
- public ITupleParser createTupleParser(IHyracksTaskContext ctx) throws HyracksDataException {
- return new RateControlledTupleParser(ctx, recordType, dataParser, configuration);
- }
-
-}
-
-class RateControlledTupleParser extends AbstractTupleParser {
-
- private final IDataParser dataParser;
- private long interTupleInterval;
- private boolean delayConfigured;
- private boolean continueIngestion = true;
-
- public static final String INTER_TUPLE_INTERVAL = "tuple-interval";
-
- public RateControlledTupleParser(IHyracksTaskContext ctx, ARecordType recType, IDataParser dataParser,
- Map<String, Object> configuration) throws HyracksDataException {
- super(ctx, recType);
- this.dataParser = dataParser;
- String propValue = (String) configuration.get(INTER_TUPLE_INTERVAL);
- if (propValue != null) {
- interTupleInterval = Long.parseLong(propValue);
- } else {
- interTupleInterval = 0;
- }
- delayConfigured = interTupleInterval != 0;
- }
-
- public void setInterTupleInterval(long val) {
- this.interTupleInterval = val;
- this.delayConfigured = val > 0;
- }
-
- public void stop() {
- continueIngestion = false;
- }
-
- @Override
- public IDataParser getDataParser() {
- return dataParser;
- }
-
- @Override
- public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
-
- appender.reset(frame, true);
- IDataParser parser = getDataParser();
- try {
- parser.initialize(in, recType, true);
- while (continueIngestion) {
- tb.reset();
- if (!parser.parse(tb.getDataOutput())) {
- break;
- }
- tb.addFieldEndOffset();
- if (delayConfigured) {
- Thread.sleep(interTupleInterval);
- }
- addTupleToFrame(writer);
- }
- if (appender.getTupleCount() > 0) {
- FrameUtils.flushFrame(frame, writer);
- }
- } catch (AsterixException ae) {
- throw new HyracksDataException(ae);
- } catch (IOException ioe) {
- throw new HyracksDataException(ioe);
- } catch (InterruptedException ie) {
- throw new HyracksDataException(ie);
- }
- }
}
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
index bf3c086..40bc814 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
@@ -14,14 +14,32 @@
*/
package edu.uci.ics.asterix.tools.external.data;
+import java.io.IOException;
+import java.io.InputStream;
import java.util.Map;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
+import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
+import edu.uci.ics.asterix.external.adapter.factory.NCFileSystemAdapterFactory;
+import edu.uci.ics.asterix.external.adapter.factory.StreamBasedAdapterFactory;
import edu.uci.ics.asterix.external.dataset.adapter.FileSystemBasedAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.runtime.operators.file.ADMDataParser;
+import edu.uci.ics.asterix.runtime.operators.file.AbstractTupleParser;
+import edu.uci.ics.asterix.runtime.operators.file.DelimitedDataParser;
+import edu.uci.ics.asterix.runtime.operators.file.IDataParser;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
/**
* Factory class for creating @see{RateControllerFileSystemBasedAdapter} The
@@ -29,38 +47,25 @@
* on the local file system or on HDFS. The feed ends when the content of the
* source file has been ingested.
*/
-public class RateControlledFileSystemBasedAdapterFactory implements IGenericDatasetAdapterFactory {
+public class RateControlledFileSystemBasedAdapterFactory extends StreamBasedAdapterFactory implements
+ IGenericAdapterFactory {
private static final long serialVersionUID = 1L;
-
+
public static final String KEY_FILE_SYSTEM = "fs";
public static final String LOCAL_FS = "localfs";
public static final String HDFS = "hdfs";
public static final String KEY_PATH = "path";
public static final String KEY_FORMAT = "format";
- private IGenericDatasetAdapterFactory adapterFactory;
+ private IGenericAdapterFactory adapterFactory;
private String format;
- private boolean setup = false;
+ private Map<String, String> configuration;
+ private ARecordType atype;
@Override
- public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType type) throws Exception {
- if (!setup) {
- checkRequiredArgs(configuration);
- String fileSystem = (String) configuration.get(KEY_FILE_SYSTEM);
- String adapterFactoryClass = null;
- if (fileSystem.equalsIgnoreCase(LOCAL_FS)) {
- adapterFactoryClass = "edu.uci.ics.asterix.external.adapter.factory.NCFileSystemAdapterFactory";
- } else if (fileSystem.equals(HDFS)) {
- adapterFactoryClass = "edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory";
- } else {
- throw new AsterixException("Unsupported file system type " + fileSystem);
- }
- format = (String) configuration.get(KEY_FORMAT);
- adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClass).newInstance();
- setup = true;
- }
- return new RateControlledFileSystemBasedAdapter((ARecordType) type, configuration,
- (FileSystemBasedAdapter) adapterFactory.createAdapter(configuration, type), format);
+ public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
+ FileSystemBasedAdapter coreAdapter = (FileSystemBasedAdapter) adapterFactory.createAdapter(ctx);
+ return new RateControlledFileSystemBasedAdapter(atype, configuration, coreAdapter, format, parserFactory, ctx);
}
@Override
@@ -68,11 +73,11 @@
return "file_feed";
}
- private void checkRequiredArgs(Map<String, Object> configuration) throws Exception {
+ private void checkRequiredArgs(Map<String, String> configuration) throws Exception {
if (configuration.get(KEY_FILE_SYSTEM) == null) {
throw new Exception("File system type not specified. (fs=?) File system could be 'localfs' or 'hdfs'");
}
- if (configuration.get(IGenericDatasetAdapterFactory.KEY_TYPE_NAME) == null) {
+ if (configuration.get(KEY_SOURCE_DATATYPE) == null) {
throw new Exception("Record type not specified (output-type-name=?)");
}
if (configuration.get(KEY_PATH) == null) {
@@ -83,4 +88,186 @@
}
}
+ @Override
+ public AdapterType getAdapterType() {
+ return AdapterType.GENERIC;
+ }
+
+ @Override
+ public SupportedOperation getSupportedOperations() {
+ return SupportedOperation.READ;
+ }
+
+ @Override
+ public void configure(Map<String, String> configuration, ARecordType recordType) throws Exception {
+ this.configuration = configuration;
+ checkRequiredArgs(configuration);
+ String fileSystem = (String) configuration.get(KEY_FILE_SYSTEM);
+ String adapterFactoryClass = null;
+ if (fileSystem.equalsIgnoreCase(LOCAL_FS)) {
+ adapterFactoryClass = NCFileSystemAdapterFactory.class.getName();
+ } else if (fileSystem.equals(HDFS)) {
+ adapterFactoryClass = HDFSAdapterFactory.class.getName();
+ } else {
+ throw new AsterixException("Unsupported file system type " + fileSystem);
+ }
+ format = configuration.get(KEY_FORMAT);
+ adapterFactory = (IGenericAdapterFactory) Class.forName(adapterFactoryClass).newInstance();
+ adapterFactory.configure(configuration, recordType);
+
+ atype = (ARecordType) recordType;
+ configureFormat();
+ }
+
+ @Override
+ public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+ return adapterFactory.getPartitionConstraint();
+ }
+
+ private void configureFormat() throws AsterixException {
+ switch (format) {
+ case FORMAT_ADM:
+ parserFactory = new RateControlledTupleParserFactory(atype, configuration);
+ break;
+
+ case FORMAT_DELIMITED_TEXT:
+ String delimiterValue = (String) configuration.get(KEY_DELIMITER);
+ if (delimiterValue != null && delimiterValue.length() > 1) {
+ throw new AsterixException("improper delimiter");
+ }
+ IValueParserFactory[] valueParserFactories = getValueParserFactories(atype);
+ parserFactory = new RateControlledTupleParserFactory(atype, valueParserFactories,
+ delimiterValue.charAt(0), configuration);
+ break;
+ }
+ }
+
+ protected IValueParserFactory[] getValueParserFactories(ARecordType recordType) throws AsterixException {
+ int n = recordType.getFieldTypes().length;
+ IValueParserFactory[] fieldParserFactories = new IValueParserFactory[n];
+ for (int i = 0; i < n; i++) {
+ ATypeTag tag = recordType.getFieldTypes()[i].getTypeTag();
+ IValueParserFactory vpf = typeToValueParserFactMap.get(tag);
+ if (vpf == null) {
+ throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
+ }
+ fieldParserFactories[i] = vpf;
+
+ }
+ return fieldParserFactories;
+ }
+
+}
+
+class RateControlledTupleParserFactory implements ITupleParserFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private final ARecordType recordType;
+ private final Map<String, String> configuration;
+ private IValueParserFactory[] valueParserFactories;
+ private char delimiter;
+ private final ParserType parserType;
+
+ public enum ParserType {
+ ADM,
+ DELIMITED_DATA
+ }
+
+ public RateControlledTupleParserFactory(ARecordType recordType, IValueParserFactory[] valueParserFactories,
+ char fieldDelimiter, Map<String, String> configuration) {
+ this.recordType = recordType;
+ this.valueParserFactories = valueParserFactories;
+ this.delimiter = fieldDelimiter;
+ this.configuration = configuration;
+ this.parserType = ParserType.DELIMITED_DATA;
+ }
+
+ public RateControlledTupleParserFactory(ARecordType recordType, Map<String, String> configuration) {
+ this.recordType = recordType;
+ this.configuration = configuration;
+ this.parserType = ParserType.ADM;
+ }
+
+ @Override
+ public ITupleParser createTupleParser(IHyracksTaskContext ctx) throws HyracksDataException {
+ IDataParser dataParser = null;
+ switch (parserType) {
+ case ADM:
+ dataParser = new ADMDataParser();
+ break;
+ case DELIMITED_DATA:
+ dataParser = new DelimitedDataParser(recordType, valueParserFactories, delimiter);
+ break;
+ }
+ return new RateControlledTupleParser(ctx, recordType, dataParser, configuration);
+ }
+
+}
+
+class RateControlledTupleParser extends AbstractTupleParser {
+
+ private final IDataParser dataParser;
+ private long interTupleInterval;
+ private boolean delayConfigured;
+ private boolean continueIngestion = true;
+
+ public static final String INTER_TUPLE_INTERVAL = "tuple-interval";
+
+ public RateControlledTupleParser(IHyracksTaskContext ctx, ARecordType recType, IDataParser dataParser,
+ Map<String, String> configuration) throws HyracksDataException {
+ super(ctx, recType);
+ this.dataParser = dataParser;
+ String propValue = configuration.get(INTER_TUPLE_INTERVAL);
+ if (propValue != null) {
+ interTupleInterval = Long.parseLong(propValue);
+ } else {
+ interTupleInterval = 0;
+ }
+ delayConfigured = interTupleInterval != 0;
+ }
+
+ public void setInterTupleInterval(long val) {
+ this.interTupleInterval = val;
+ this.delayConfigured = val > 0;
+ }
+
+ public void stop() {
+ continueIngestion = false;
+ }
+
+ @Override
+ public IDataParser getDataParser() {
+ return dataParser;
+ }
+
+ @Override
+ public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
+
+ appender.reset(frame, true);
+ IDataParser parser = getDataParser();
+ try {
+ parser.initialize(in, recType, true);
+ while (continueIngestion) {
+ tb.reset();
+ if (!parser.parse(tb.getDataOutput())) {
+ break;
+ }
+ tb.addFieldEndOffset();
+ if (delayConfigured) {
+ Thread.sleep(interTupleInterval);
+ }
+ addTupleToFrame(writer);
+ }
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(frame, writer);
+ }
+ } catch (AsterixException ae) {
+ throw new HyracksDataException(ae);
+ } catch (IOException ioe) {
+ throw new HyracksDataException(ioe);
+ } catch (InterruptedException ie) {
+ throw new HyracksDataException(ie);
+ }
+ }
}
\ No newline at end of file
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SyntheticTwitterFeedAdapter.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SyntheticTwitterFeedAdapter.java
new file mode 100644
index 0000000..d26e764
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SyntheticTwitterFeedAdapter.java
@@ -0,0 +1,214 @@
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.util.List;
+import java.util.Map;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import edu.uci.ics.asterix.external.dataset.adapter.IPullBasedFeedClient;
+import edu.uci.ics.asterix.external.dataset.adapter.PullBasedAdapter;
+import edu.uci.ics.asterix.external.dataset.adapter.PullBasedFeedClient;
+import edu.uci.ics.asterix.om.base.AMutableDateTime;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutablePoint;
+import edu.uci.ics.asterix.om.base.AMutableRecord;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.AMutableUnorderedList;
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.InitializationInfo;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.Message;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.TweetMessage;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.TweetMessageIterator;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * TPS can be configured between 1 and 20,000
+ *
+ * @author ramang
+ */
+public class SyntheticTwitterFeedAdapter extends PullBasedAdapter {
+
+ private static final long serialVersionUID = 1L;
+
+ private static final Logger LOGGER = Logger.getLogger(SyntheticTwitterFeedAdapter.class.getName());
+
+ private Map<String, String> configuration;
+
+ public SyntheticTwitterFeedAdapter(Map<String, String> configuration, ARecordType outputType,
+ IHyracksTaskContext ctx) throws AsterixException {
+ super(configuration, ctx);
+ this.configuration = configuration;
+ this.adapterOutputType = outputType;
+ }
+
+ @Override
+ public IPullBasedFeedClient getFeedClient(int partition) throws Exception {
+ return new SyntheticTwitterFeedClient(configuration, adapterOutputType, partition);
+ }
+
+ private static class SyntheticTwitterFeedClient extends PullBasedFeedClient implements IPullBasedFeedClient {
+
+ private static final Logger LOGGER = Logger.getLogger(SyntheticTwitterFeedClient.class.getName());
+
+ public static final String KEY_DURATION = "duration";
+ public static final String KEY_TPS = "tps";
+ public static final String KEY_EXCEPTION_PERIOD = "exception-period";
+
+ private int duration;
+ private long tweetInterval;
+ private int numTweetsBeforeDelay;
+ private TweetMessageIterator tweetIterator = null;
+ private long exeptionInterval;
+
+ private IAObject[] mutableFields;
+ private ARecordType outputRecordType;
+ private int partition;
+ private int tweetCount = 0;
+ private int tweetCountBeforeException = 0;
+ private int exceptionPeriod = -1;
+
+ public SyntheticTwitterFeedClient(Map<String, String> configuration, ARecordType outputRecordType, int partition)
+ throws AsterixException {
+ this.outputRecordType = outputRecordType;
+ String value = (String) configuration.get(KEY_DURATION);
+ duration = value != null ? Integer.parseInt(value) : 60;
+ initializeTweetRate((String) configuration.get(KEY_TPS));
+ value = (String) configuration.get(KEY_EXCEPTION_PERIOD);
+ if (value != null) {
+ exceptionPeriod = Integer.parseInt(value);
+ }
+
+ InitializationInfo info = new InitializationInfo();
+ info.timeDurationInSecs = duration;
+ DataGenerator.initialize(info);
+ tweetIterator = new TweetMessageIterator(duration);
+ initialize();
+ }
+
+ private void initializeTweetRate(String tps) {
+ numTweetsBeforeDelay = 0;
+ if (tps == null) {
+ tweetInterval = 0;
+ } else {
+ int val = Integer.parseInt(tps);
+ double interval = new Double(((double) 1000 / val));
+ if (interval > 1) {
+ tweetInterval = (long) interval;
+ numTweetsBeforeDelay = 1;
+ } else {
+ tweetInterval = 1;
+ Double numTweets = new Double(1 / interval);
+ if (numTweets.intValue() != numTweets) {
+ tweetInterval = 10;
+ numTweetsBeforeDelay = (new Double(10 * numTweets * 1.4)).intValue();
+ } else {
+ numTweetsBeforeDelay = new Double((numTweets * 1.4)).intValue();
+ }
+ }
+ }
+
+ }
+
+ private void writeTweet(TweetMessage next) {
+
+ // tweet id
+ LOGGER.info("Generating next tweet");
+ ((AMutableString) mutableFields[0]).setValue(next.getTweetid());
+ mutableRecord.setValueAtPos(0, mutableFields[0]);
+
+ // user
+ AMutableRecord userRecord = ((AMutableRecord) mutableFields[1]);
+ ((AMutableString) userRecord.getValueByPos(0)).setValue(next.getUser().getScreenName());
+ ((AMutableString) userRecord.getValueByPos(1)).setValue("en");
+ ((AMutableInt32) userRecord.getValueByPos(2)).setValue(next.getUser().getFriendsCount());
+ ((AMutableInt32) userRecord.getValueByPos(3)).setValue(next.getUser().getStatusesCount());
+ ((AMutableString) userRecord.getValueByPos(4)).setValue(next.getUser().getName());
+ ((AMutableInt32) userRecord.getValueByPos(5)).setValue(next.getUser().getFollowersCount());
+ mutableRecord.setValueAtPos(1, userRecord);
+
+ // location
+ ((AMutablePoint) mutableFields[2]).setValue(next.getSenderLocation().getLatitude(), next
+ .getSenderLocation().getLongitude());
+ mutableRecord.setValueAtPos(2, mutableFields[2]);
+
+ // time
+ ((AMutableDateTime) mutableFields[3]).setValue(next.getSendTime().getChrononTime());
+ mutableRecord.setValueAtPos(3, mutableFields[3]);
+
+ // referred topics
+ ((AMutableUnorderedList) mutableFields[4]).clear();
+ List<String> referredTopics = next.getReferredTopics();
+ for (String topic : referredTopics) {
+ ((AMutableUnorderedList) mutableFields[4]).add(new AMutableString(topic));
+ }
+ mutableRecord.setValueAtPos(4, mutableFields[4]);
+
+ // text
+ Message m = next.getMessageText();
+ char[] content = m.getMessage();
+ String tweetText = new String(content, 0, m.getLength());
+ ((AMutableString) mutableFields[5]).setValue(tweetText);
+ mutableRecord.setValueAtPos(5, mutableFields[5]);
+ LOGGER.info(tweetText);
+
+ }
+
+ @Override
+ public void resetOnFailure(Exception e) throws AsterixException {
+ // TODO Auto-generated method stub
+
+ }
+
+ @Override
+ public boolean alter(Map<String, String> configuration) {
+ // TODO Auto-generated method stub
+ return false;
+ }
+
+ @Override
+ public InflowState setNextRecord() throws Exception {
+ boolean moreData = tweetIterator.hasNext();
+ if (!moreData) {
+ return InflowState.NO_MORE_DATA;
+ }
+ writeTweet(tweetIterator.next());
+ if (tweetInterval != 0) {
+ tweetCount++;
+ if (tweetCount == numTweetsBeforeDelay) {
+ Thread.sleep(tweetInterval);
+ tweetCount = 0;
+ }
+ }
+ tweetCountBeforeException++;
+
+ if (tweetCountBeforeException == exceptionPeriod) {
+ tweetCountBeforeException = 0;
+ throw new AsterixException("Delibrate exception");
+ }
+ return InflowState.DATA_AVAILABLE;
+ }
+
+ private void initialize() throws AsterixException {
+ ARecordType userRecordType = (ARecordType) outputRecordType.getFieldTypes()[1];
+ IAObject[] userMutableFields = new IAObject[] { new AMutableString(""), new AMutableString(""),
+ new AMutableInt32(0), new AMutableInt32(0), new AMutableString(""), new AMutableInt32(0) };
+ AUnorderedListType unorderedListType = new AUnorderedListType(BuiltinType.ASTRING, "referred-topics");
+ mutableFields = new IAObject[] { new AMutableString(""),
+ new AMutableRecord(userRecordType, userMutableFields), new AMutablePoint(0, 0),
+ new AMutableDateTime(0), new AMutableUnorderedList(unorderedListType), new AMutableString("") };
+ recordSerDe = new ARecordSerializerDeserializer(outputRecordType);
+ mutableRecord = new AMutableRecord(outputRecordType, mutableFields);
+
+ }
+
+ @Override
+ public void stop() {
+ // TODO Auto-generated method stub
+
+ }
+ }
+}
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SyntheticTwitterFeedAdapterFactory.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SyntheticTwitterFeedAdapterFactory.java
new file mode 100644
index 0000000..11cbe54
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SyntheticTwitterFeedAdapterFactory.java
@@ -0,0 +1,141 @@
+/*
+x * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.NodeGroup;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * Factory class for creating @see{RateControllerFileSystemBasedAdapter} The
+ * adapter simulates a feed from the contents of a source file. The file can be
+ * on the local file system or on HDFS. The feed ends when the content of the
+ * source file has been ingested.
+ */
+public class SyntheticTwitterFeedAdapterFactory implements ITypedAdapterFactory {
+
+ /**
+ *
+ */
+ private static final long serialVersionUID = 1L;
+
+ private Map<String, String> configuration;
+
+ private static final String KEY_DATAVERSE_DATASET = "dataverse-dataset";
+
+ private static final ARecordType outputType = initOutputType();
+
+ @Override
+ public String getName() {
+ return "synthetic_twitter_feed";
+ }
+
+ @Override
+ public AdapterType getAdapterType() {
+ return AdapterType.TYPED;
+ }
+
+ @Override
+ public SupportedOperation getSupportedOperations() {
+ return SupportedOperation.READ;
+ }
+
+ @Override
+ public void configure(Map<String, String> configuration) throws Exception {
+ this.configuration = configuration;
+ }
+
+ @Override
+ public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+ String dvds = (String) configuration.get(KEY_DATAVERSE_DATASET);
+ String[] components = dvds.split(":");
+ String dataverse = components[0];
+ String dataset = components[1];
+ MetadataTransactionContext ctx = null;
+ NodeGroup ng = null;
+ try {
+ ctx = MetadataManager.INSTANCE.beginTransaction();
+ Dataset ds = MetadataManager.INSTANCE.getDataset(ctx, dataverse, dataset);
+ String nodegroupName = ((InternalDatasetDetails) ds.getDatasetDetails()).getNodeGroupName();
+ ng = MetadataManager.INSTANCE.getNodegroup(ctx, nodegroupName);
+ MetadataManager.INSTANCE.commitTransaction(ctx);
+ } catch (Exception e) {
+ MetadataManager.INSTANCE.abortTransaction(ctx);
+ throw e;
+ }
+ List<String> storageNodes = ng.getNodeNames();
+ Set<String> nodes = AsterixAppContextInfo.getInstance().getMetadataProperties().getNodeNames();
+ String ingestionLocation = null;
+ if (nodes.size() > storageNodes.size()) {
+ nodes.removeAll(storageNodes);
+ }
+ String[] nodesArray = nodes.toArray(new String[] {});
+ Random r = new Random();
+ ingestionLocation = nodesArray[r.nextInt(nodes.size())];
+ return new AlgebricksAbsolutePartitionConstraint(new String[] { ingestionLocation });
+ }
+
+ @Override
+ public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
+ return new SyntheticTwitterFeedAdapter(configuration, outputType, ctx);
+ }
+
+ @Override
+ public ARecordType getAdapterOutputType() {
+ return outputType;
+ }
+
+ private static ARecordType initOutputType() {
+ ARecordType outputType = null;
+ try {
+ String[] userFieldNames = new String[] { "screen-name", "lang", "friends_count", "statuses_count", "name",
+ "followers_count" };
+
+ IAType[] userFieldTypes = new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.AINT32,
+ BuiltinType.AINT32, BuiltinType.ASTRING, BuiltinType.AINT32 };
+ ARecordType userRecordType = new ARecordType("TwitterUserType", userFieldNames, userFieldTypes, false);
+
+ String[] fieldNames = new String[] { "tweetid", "user", "sender-location", "send-time", "referred-topics",
+ "message-text" };
+
+ AUnorderedListType unorderedListType = new AUnorderedListType(BuiltinType.ASTRING, "referred-topics");
+ IAType[] fieldTypes = new IAType[] { BuiltinType.ASTRING, userRecordType, BuiltinType.APOINT,
+ BuiltinType.ADATETIME, unorderedListType, BuiltinType.ASTRING };
+ outputType = new ARecordType("TweetMessageType", fieldNames, fieldTypes, false);
+
+ } catch (AsterixException e) {
+ throw new IllegalStateException("Unable to initialize output type");
+ }
+ return outputType;
+ }
+}
\ No newline at end of file
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TweetGenerator.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TweetGenerator.java
new file mode 100644
index 0000000..9d47de2
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TweetGenerator.java
@@ -0,0 +1,213 @@
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.List;
+import java.util.Map;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import edu.uci.ics.asterix.external.dataset.adapter.IPullBasedFeedClient;
+import edu.uci.ics.asterix.external.dataset.adapter.PullBasedFeedClient;
+import edu.uci.ics.asterix.om.base.AMutableDateTime;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutablePoint;
+import edu.uci.ics.asterix.om.base.AMutableRecord;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.AMutableUnorderedList;
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.InitializationInfo;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.Message;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.TweetMessage;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator.TweetMessageIterator;
+
+public class TweetGenerator extends PullBasedFeedClient implements IPullBasedFeedClient {
+
+ private static final Logger LOGGER = Logger.getLogger(TweetGenerator.class.getName());
+
+ public static final String KEY_DURATION = "duration";
+ public static final String KEY_TPS = "tps";
+ public static final String KEY_EXCEPTION_PERIOD = "exception-period";
+ public static final String OUTPUT_FORMAT = "output-format";
+
+ public static final String OUTPUT_FORMAT_ARECORD = "arecord";
+ public static final String OUTPUT_FORMAT_ADM_STRING = "adm-string";
+
+ private int duration;
+ private long tweetInterval;
+ private int numTweetsBeforeDelay;
+ private TweetMessageIterator tweetIterator = null;
+ private long exeptionInterval;
+
+ private IAObject[] mutableFields;
+ private ARecordType outputRecordType;
+ private int partition;
+ private int tweetCount = 0;
+ private int tweetCountBeforeException = 0;
+ private int exceptionPeriod = -1;
+ private boolean isOutputFormatRecord = false;
+ private byte[] EOL = "\n".getBytes();
+ private OutputStream os;
+
+ public TweetGenerator(Map<String, String> configuration, ARecordType outputRecordType, int partition, String format)
+ throws AsterixException {
+ this.outputRecordType = outputRecordType;
+ String value = configuration.get(KEY_DURATION);
+ duration = value != null ? Integer.parseInt(value) : 60;
+ initializeTweetRate(configuration.get(KEY_TPS));
+ value = configuration.get(KEY_EXCEPTION_PERIOD);
+ if (value != null) {
+ exceptionPeriod = Integer.parseInt(value);
+ }
+ isOutputFormatRecord = format.equalsIgnoreCase(OUTPUT_FORMAT_ARECORD);
+ InitializationInfo info = new InitializationInfo();
+ info.timeDurationInSecs = duration;
+ DataGenerator.initialize(info);
+ tweetIterator = new TweetMessageIterator(duration);
+ initialize();
+ }
+
+ private void initializeTweetRate(String tps) {
+ numTweetsBeforeDelay = 0;
+ if (tps == null) {
+ tweetInterval = 0;
+ } else {
+ int val = Integer.parseInt(tps);
+ double interval = new Double(((double) 1000 / val));
+ if (interval > 1) {
+ tweetInterval = (long) interval;
+ numTweetsBeforeDelay = 1;
+ } else {
+ tweetInterval = 1;
+ Double numTweets = new Double(1 / interval);
+ if (numTweets.intValue() != numTweets) {
+ tweetInterval = 10;
+ numTweetsBeforeDelay = (new Double(10 * numTweets * 1.4)).intValue();
+ } else {
+ numTweetsBeforeDelay = new Double((numTweets * 1.4)).intValue();
+ }
+ }
+ }
+
+ }
+
+ private void writeTweetString(TweetMessage next) throws IOException {
+ String tweet = next.toString();
+ os.write(tweet.getBytes());
+ os.write(EOL);
+ /*
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(tweet);
+ }*/
+ }
+
+ private void writeTweetRecord(TweetMessage next) {
+
+ //tweet id
+ ((AMutableString) mutableFields[0]).setValue(next.getTweetid());
+ mutableRecord.setValueAtPos(0, mutableFields[0]);
+
+ // user
+ AMutableRecord userRecord = ((AMutableRecord) mutableFields[1]);
+ ((AMutableString) userRecord.getValueByPos(0)).setValue(next.getUser().getScreenName());
+ ((AMutableString) userRecord.getValueByPos(1)).setValue("en");
+ ((AMutableInt32) userRecord.getValueByPos(2)).setValue(next.getUser().getFriendsCount());
+ ((AMutableInt32) userRecord.getValueByPos(3)).setValue(next.getUser().getStatusesCount());
+ ((AMutableString) userRecord.getValueByPos(4)).setValue(next.getUser().getName());
+ ((AMutableInt32) userRecord.getValueByPos(5)).setValue(next.getUser().getFollowersCount());
+ mutableRecord.setValueAtPos(1, userRecord);
+
+ // location
+ ((AMutablePoint) mutableFields[2]).setValue(next.getSenderLocation().getLatitude(), next.getSenderLocation()
+ .getLongitude());
+ mutableRecord.setValueAtPos(2, mutableFields[2]);
+
+ // time
+ ((AMutableDateTime) mutableFields[3]).setValue(next.getSendTime().getChrononTime());
+ mutableRecord.setValueAtPos(3, mutableFields[3]);
+
+ // referred topics
+ ((AMutableUnorderedList) mutableFields[4]).clear();
+ List<String> referredTopics = next.getReferredTopics();
+ for (String topic : referredTopics) {
+ ((AMutableUnorderedList) mutableFields[4]).add(new AMutableString(topic));
+ }
+ mutableRecord.setValueAtPos(4, mutableFields[4]);
+
+ // text
+ Message m = next.getMessageText();
+ char[] content = m.getMessage();
+ String tweetText = new String(content, 0, m.getLength());
+ ((AMutableString) mutableFields[5]).setValue(tweetText);
+ mutableRecord.setValueAtPos(5, mutableFields[5]);
+ LOGGER.info(tweetText);
+
+ }
+
+ @Override
+ public void resetOnFailure(Exception e) throws AsterixException {
+ // TODO Auto-generated method stub
+
+ }
+
+ @Override
+ public boolean alter(Map<String, String> configuration) {
+ // TODO Auto-generated method stub
+ return false;
+ }
+
+ @Override
+ public InflowState setNextRecord() throws Exception {
+ boolean moreData = tweetIterator.hasNext();
+ if (!moreData) {
+ return InflowState.NO_MORE_DATA;
+ }
+ TweetMessage msg = tweetIterator.next();
+ if (isOutputFormatRecord) {
+ writeTweetRecord(msg);
+ } else {
+ writeTweetString(msg);
+ }
+ if (tweetInterval != 0) {
+ tweetCount++;
+ if (tweetCount == numTweetsBeforeDelay) {
+ Thread.sleep(tweetInterval);
+ tweetCount = 0;
+ }
+ }
+ tweetCountBeforeException++;
+
+ if (tweetCountBeforeException == exceptionPeriod) {
+ tweetCountBeforeException = 0;
+ throw new AsterixException("Delibrate exception");
+ }
+ return InflowState.DATA_AVAILABLE;
+ }
+
+ private void initialize() throws AsterixException {
+ ARecordType userRecordType = (ARecordType) outputRecordType.getFieldTypes()[1];
+ IAObject[] userMutableFields = new IAObject[] { new AMutableString(""), new AMutableString(""),
+ new AMutableInt32(0), new AMutableInt32(0), new AMutableString(""), new AMutableInt32(0) };
+ AUnorderedListType unorderedListType = new AUnorderedListType(BuiltinType.ASTRING, "referred-topics");
+ mutableFields = new IAObject[] { new AMutableString(""), new AMutableRecord(userRecordType, userMutableFields),
+ new AMutablePoint(0, 0), new AMutableDateTime(0), new AMutableUnorderedList(unorderedListType),
+ new AMutableString("") };
+ recordSerDe = new ARecordSerializerDeserializer(outputRecordType);
+ mutableRecord = new AMutableRecord(outputRecordType, mutableFields);
+
+ }
+
+ @Override
+ public void stop() {
+ // TODO Auto-generated method stub
+
+ }
+
+ public void setOutputStream(OutputStream os) {
+ this.os = os;
+ }
+}
\ No newline at end of file
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TweetGenerator2.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TweetGenerator2.java
new file mode 100644
index 0000000..6d0b1f9
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TweetGenerator2.java
@@ -0,0 +1,169 @@
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.tools.external.data.DataGenerator2.InitializationInfo;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator2.TweetMessage;
+import edu.uci.ics.asterix.tools.external.data.DataGenerator2.TweetMessageIterator;
+
+public class TweetGenerator2 {
+
+ private static final Logger LOGGER = Logger.getLogger(TweetGenerator.class.getName());
+
+ public static final String KEY_DURATION = "duration";
+ public static final String KEY_TPS = "tps";
+ public static final String KEY_MIN_TPS = "tps-min";
+ public static final String KEY_MAX_TPS = "tps-max";
+
+
+ public static final String KEY_TPUT_DURATION = "tput-duration";
+
+ public static final String OUTPUT_FORMAT = "output-format";
+
+ public static final String OUTPUT_FORMAT_ARECORD = "arecord";
+ public static final String OUTPUT_FORMAT_ADM_STRING = "adm-string";
+
+ private int duration;
+ private long tweetInterval;
+ private int numTweetsBeforeDelay;
+ private TweetMessageIterator tweetIterator = null;
+ private long exeptionInterval;
+
+
+
+ private int partition;
+ private int tweetCount = 0;
+ private int frameTweetCount = 0;
+ private int numFlushedTweets = 0;
+
+ public int getTweetCount() {
+ return tweetCount;
+ }
+
+ private int exceptionPeriod = -1;
+ private boolean isOutputFormatRecord = false;
+ private byte[] EOL = "\n".getBytes();
+ private OutputStream os;
+ private DataGenerator2 dataGenerator = null;
+ private ByteBuffer outputBuffer = ByteBuffer.allocate(32 * 1024);
+ private int flushedTweetCount = 0;
+
+ public TweetGenerator2(Map<String, String> configuration, int partition, String format) throws Exception {
+ String value = configuration.get(KEY_DURATION);
+ duration = value != null ? Integer.parseInt(value) : 60;
+ initializeTweetRate(configuration.get(KEY_TPS));
+ if (value != null) {
+ exceptionPeriod = Integer.parseInt(value);
+ }
+
+ isOutputFormatRecord = format.equalsIgnoreCase(OUTPUT_FORMAT_ARECORD);
+ InitializationInfo info = new InitializationInfo();
+ info.timeDurationInSecs = duration;
+ dataGenerator = new DataGenerator2(info);
+ tweetIterator = dataGenerator.new TweetMessageIterator(duration);
+ }
+
+ private void initializeTweetRate(String tps) {
+ numTweetsBeforeDelay = 0;
+ if (tps == null) {
+ tweetInterval = 0;
+ } else {
+ int val = Integer.parseInt(tps);
+ double interval = new Double(((double) 1000 / val));
+ if (interval > 1) {
+ tweetInterval = (long) interval;
+ numTweetsBeforeDelay = 1;
+ } else {
+ tweetInterval = 1;
+ Double numTweets = new Double(1 / interval);
+ if (numTweets.intValue() != numTweets) {
+ tweetInterval = 5;
+ numTweetsBeforeDelay = (new Double(10 * numTweets * 1)).intValue();
+ } else {
+ numTweetsBeforeDelay = new Double((numTweets * 1)).intValue();
+ }
+ }
+ }
+
+ }
+
+ private void writeTweetString(TweetMessage next) throws IOException {
+ String tweet = next.toString() + "\n";
+ tweetCount++;
+ byte[] b = tweet.getBytes();
+ if (outputBuffer.position() + b.length > outputBuffer.limit()) {
+ flush();
+ numFlushedTweets += frameTweetCount;
+ frameTweetCount = 0;
+ flushedTweetCount += tweetCount - 1;
+ outputBuffer.put(tweet.getBytes());
+ frameTweetCount++;
+
+ } else {
+ outputBuffer.put(tweet.getBytes());
+ frameTweetCount++;
+ }
+ }
+
+ public int getNumFlushedTweets() {
+ return numFlushedTweets;
+ }
+
+ public int getFrameTweetCount() {
+ return frameTweetCount;
+ }
+
+ private void flush() throws IOException {
+ outputBuffer.flip();
+ os.write(outputBuffer.array(), 0, outputBuffer.limit());
+ outputBuffer.position(0);
+ outputBuffer.limit(32 * 1024);
+ }
+
+ private void writeTweetRecord(TweetMessage next) {
+ throw new UnsupportedOperationException("Invalid format");
+ }
+
+ public boolean setNextRecord() throws Exception {
+ boolean moreData = tweetIterator.hasNext();
+ if (!moreData) {
+ return false;
+ }
+ TweetMessage msg = tweetIterator.next();
+ if (isOutputFormatRecord) {
+ writeTweetRecord(msg);
+ } else {
+ writeTweetString(msg);
+ }
+ if (tweetInterval != 0) {
+ tweetCount++;
+ if (tweetCount == numTweetsBeforeDelay) {
+ Thread.sleep(tweetInterval);
+ tweetCount = 0;
+ }
+ }
+ return true;
+ }
+
+ public boolean setNextRecordBatch(int numTweetsInBatch) throws Exception {
+ int count = 0;
+ // if (tweetIterator.hasNext()) {
+ while (count < numTweetsInBatch) {
+ writeTweetString(tweetIterator.next());
+ count++;
+ }
+ // } else {
+ // System.out.println("Flushing last batch, count so far:" + tweetCount);
+ // flush();
+ /// }
+ return true;
+ }
+
+ public void setOutputStream(OutputStream os) {
+ this.os = os;
+ }
+}
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapter.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapter.java
new file mode 100644
index 0000000..5739c70
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapter.java
@@ -0,0 +1,299 @@
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.UnknownHostException;
+import java.util.Date;
+import java.util.Map;
+import java.util.Random;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.external.dataset.adapter.StreamBasedAdapter;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+/**
+ * TPS can be configured between 1 and 20,000
+ *
+ */
+public class TwitterFirehoseFeedAdapter extends StreamBasedAdapter implements IFeedAdapter {
+
+ private static final long serialVersionUID = 1L;
+
+ private static final Logger LOGGER = Logger.getLogger(TwitterFirehoseFeedAdapter.class.getName());
+
+ private final TwitterServer twitterServer;
+
+ private TwitterClient twitterClient;
+
+ private static final String LOCALHOST = "127.0.0.1";
+ private static final int PORT = 2909;
+
+ private ExecutorService executorService = Executors.newCachedThreadPool();
+
+ public TwitterFirehoseFeedAdapter(Map<String, String> configuration, ITupleParserFactory parserFactory,
+ ARecordType outputtype, IHyracksTaskContext ctx) throws Exception {
+ super(parserFactory, outputtype, ctx);
+ this.twitterServer = new TwitterServer(configuration, outputtype, executorService);
+ this.twitterClient = new TwitterClient(twitterServer.getPort());
+ }
+
+ @Override
+ public void start(int partition, IFrameWriter writer) throws Exception {
+ twitterServer.start();
+ twitterServer.getListener().setPartition(partition);
+ twitterClient.start();
+ super.start(partition, writer);
+ }
+
+ @Override
+ public InputStream getInputStream(int partition) throws IOException {
+ return twitterClient.getInputStream();
+ }
+
+ private static class TwitterServer {
+ private ServerSocket serverSocket;
+ private final Listener listener;
+ private int port = -1;
+ private ExecutorService executorService;
+
+ public TwitterServer(Map<String, String> configuration, ARecordType outputtype, ExecutorService executorService)
+ throws Exception {
+ int numAttempts = 0;
+ while (port < 0) {
+ try {
+ serverSocket = new ServerSocket(PORT + numAttempts);
+ port = PORT + numAttempts;
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("port: " + (PORT + numAttempts) + " unusable ");
+ }
+ numAttempts++;
+ }
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Twitter server configured to use port: " + port);
+ }
+ String dvds = configuration.get("dataverse-dataset");
+ listener = new Listener(serverSocket, configuration, outputtype, dvds);
+ this.executorService = executorService;
+ }
+
+ public Listener getListener() {
+ return listener;
+ }
+
+ public void start() {
+ executorService.execute(listener);
+ }
+
+ public void stop() throws IOException {
+ listener.stop();
+ serverSocket.close();
+ }
+
+ public int getPort() {
+ return port;
+ }
+
+ }
+
+ private static class TwitterClient {
+
+ private Socket socket;
+ private int port;
+
+ public TwitterClient(int port) throws UnknownHostException, IOException {
+ this.port = port;
+ }
+
+ public InputStream getInputStream() throws IOException {
+ return socket.getInputStream();
+ }
+
+ public void start() throws UnknownHostException, IOException {
+ socket = new Socket(LOCALHOST, port);
+ }
+
+ }
+
+ private static class Listener implements Runnable {
+
+ private final ServerSocket serverSocket;
+ private Socket socket;
+ private TweetGenerator2 tweetGenerator;
+ private boolean continuePush = true;
+ private int fixedTps = -1;
+ private int minTps = -1;
+ private int maxTps = -1;
+ private int tputDuration;
+ private int partition;
+ private Rate task;
+ private Mode mode;
+
+ public static final String KEY_MODE = "mode";
+
+ public static enum Mode {
+ AGGRESSIVE,
+ CONTROLLED,
+ }
+
+ public void setPartition(int partition) {
+ this.partition = partition;
+ task.setPartition(partition);
+ }
+
+ public Listener(ServerSocket serverSocket, Map<String, String> configuration, ARecordType outputtype,
+ String datasetName) throws Exception {
+ this.serverSocket = serverSocket;
+ this.tweetGenerator = new TweetGenerator2(configuration, 0, TweetGenerator.OUTPUT_FORMAT_ADM_STRING);
+ String value = configuration.get(KEY_MODE);
+ String confValue = null;
+ if (value != null) {
+ mode = Mode.valueOf(value.toUpperCase());
+ switch (mode) {
+ case AGGRESSIVE:
+ break;
+ case CONTROLLED:
+ confValue = configuration.get(TweetGenerator2.KEY_TPS);
+ if (confValue != null) {
+ minTps = Integer.parseInt(confValue);
+ maxTps = minTps;
+ fixedTps = minTps;
+ } else {
+ confValue = configuration.get(TweetGenerator2.KEY_MIN_TPS);
+ if (confValue != null) {
+ minTps = Integer.parseInt(confValue);
+ }
+ confValue = configuration.get(TweetGenerator2.KEY_MAX_TPS);
+ if (confValue != null) {
+ maxTps = Integer.parseInt(configuration.get(TweetGenerator2.KEY_MAX_TPS));
+ }
+
+ if (minTps < 0 || maxTps < 0 || minTps > maxTps) {
+ throw new IllegalArgumentException("Incorrect value for min/max TPS");
+ }
+ }
+
+ }
+ } else {
+ mode = Mode.AGGRESSIVE;
+ }
+
+ tputDuration = Integer.parseInt(configuration.get(TweetGenerator2.KEY_TPUT_DURATION));
+ task = new Rate(tweetGenerator, tputDuration, datasetName, partition);
+
+ }
+
+ @Override
+ public void run() {
+ while (true) {
+ try {
+ socket = serverSocket.accept();
+ OutputStream os = socket.getOutputStream();
+ tweetGenerator.setOutputStream(os);
+ boolean moreData = true;
+ Timer timer = new Timer();
+ timer.schedule(task, tputDuration * 1000, tputDuration * 1000);
+ long startBatch;
+ long endBatch;
+ Random random = new Random();
+ int tps = 0;
+ while (moreData && continuePush) {
+ if(maxTps > 0){
+ tps = minTps + random.nextInt((maxTps+1) - minTps);
+ } else {
+ tps = fixedTps;
+ }
+ startBatch = System.currentTimeMillis();
+ moreData = tweetGenerator.setNextRecordBatch(tps);
+ endBatch = System.currentTimeMillis();
+ if (mode.equals(Mode.CONTROLLED)) {
+ if (endBatch - startBatch < 1000) {
+ Thread.sleep(1000 - (endBatch - startBatch));
+ }
+ }
+ }
+ timer.cancel();
+ os.close();
+ break;
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Exception in adaptor " + e.getMessage());
+ }
+ } finally {
+ try {
+ if (socket != null && socket.isClosed()) {
+ socket.close();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Closed socket:" + socket.getPort());
+ }
+ }
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+
+ }
+ }
+ }
+
+ public void stop() {
+ continuePush = false;
+ }
+
+ private static class Rate extends TimerTask {
+
+ private TweetGenerator2 gen;
+ int prevMeasuredTweets = 0;
+ private int tputDuration;
+ private int partition;
+ private String dataset;
+
+ public Rate(TweetGenerator2 gen, int tputDuration, String dataset, int partition) {
+ this.gen = gen;
+ this.tputDuration = tputDuration;
+ this.dataset = dataset;
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning(new Date() + " " + "Dataset" + " " + "partition" + " " + "Total flushed tweets"
+ + "\t" + "intantaneous throughput");
+ }
+ }
+
+ @Override
+ public void run() {
+
+ int currentMeasureTweets = gen.getNumFlushedTweets();
+
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine(dataset + " " + partition + " " + gen.getNumFlushedTweets() + "\t"
+ + ((currentMeasureTweets - prevMeasuredTweets) / tputDuration) + " ID "
+ + Thread.currentThread().getId());
+ }
+
+ prevMeasuredTweets = currentMeasureTweets;
+
+ }
+
+ public void setPartition(int partition) {
+ this.partition = partition;
+ }
+ }
+ }
+
+ @Override
+ public void stop() throws Exception {
+ twitterServer.stop();
+ }
+
+}
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java
new file mode 100644
index 0000000..1d8a607
--- /dev/null
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java
@@ -0,0 +1,142 @@
+/*
+x * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.tools.external.data;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.external.adapter.factory.StreamBasedAdapterFactory;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * Factory class for creating @see{TwitterFirehoseFeedAdapter}.
+ * The adapter simulates a twitter firehose with tweets being "pushed" into Asterix at a configurable rate
+ * measured in terms of TPS (tweets/second). The stream of tweets lasts for a configurable duration (measured in seconds).
+ */
+public class TwitterFirehoseFeedAdapterFactory extends StreamBasedAdapterFactory implements ITypedAdapterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ /*
+ * The dataverse and dataset names for the target feed dataset. This informaiton
+ * is used in configuring partition constraints for the adapter. It is preferred that
+ * the adapter location does not coincide with a partition location for the feed dataset.
+ */
+ private static final String KEY_DATAVERSE_DATASET = "dataverse-dataset";
+
+ /*
+ * Degree of parallelism for feed ingestion activity. Defaults to 1.
+ * This builds up the count constraint for the ingestion operator.
+ */
+ private static final String KEY_INGESTION_CARDINALITY = "ingestion-cardinality";
+
+ /*
+ * The absolute locations where ingestion operator instances will be places.
+ */
+ private static final String KEY_INGESTION_LOCATIONS = "ingestion-location";
+
+ private static final ARecordType outputType = initOutputType();
+
+ @Override
+ public String getName() {
+ return "twitter_firehose";
+ }
+
+ @Override
+ public AdapterType getAdapterType() {
+ return AdapterType.TYPED;
+ }
+
+ @Override
+ public SupportedOperation getSupportedOperations() {
+ return SupportedOperation.READ;
+ }
+
+ @Override
+ public void configure(Map<String, String> configuration) throws Exception {
+ configuration.put(KEY_FORMAT, FORMAT_ADM);
+ this.configuration = configuration;
+ this.configureFormat(initOutputType());
+ }
+
+ @Override
+ public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+ String ingestionCardinalityParam = (String) configuration.get(KEY_INGESTION_CARDINALITY);
+ String ingestionLocationParam = (String) configuration.get(KEY_INGESTION_LOCATIONS);
+ String[] locations = null;
+ if (ingestionLocationParam != null) {
+ locations = ingestionLocationParam.split(",");
+ }
+ int count = locations != null ? locations.length : 1;
+ if (ingestionCardinalityParam != null) {
+ count = Integer.parseInt(ingestionCardinalityParam);
+ }
+
+ List<String> chosenLocations = new ArrayList<String>();
+ String[] availableLocations = locations != null ? locations : AsterixClusterProperties.INSTANCE
+ .getParticipantNodes().toArray(new String[] {});
+ for (int i = 0, k = 0; i < count; i++, k = (k + 1) % availableLocations.length) {
+ chosenLocations.add(availableLocations[k]);
+ }
+ return new AlgebricksAbsolutePartitionConstraint(chosenLocations.toArray(new String[] {}));
+ }
+
+ @Override
+ public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx) throws Exception {
+ return new TwitterFirehoseFeedAdapter(configuration, parserFactory, outputType, ctx);
+ }
+
+ @Override
+ public ARecordType getAdapterOutputType() {
+ return outputType;
+ }
+
+ private static ARecordType initOutputType() {
+ ARecordType outputType = null;
+ try {
+ String[] userFieldNames = new String[] { "screen-name", "lang", "friends_count", "statuses_count", "name",
+ "followers_count" };
+
+ IAType[] userFieldTypes = new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.AINT32,
+ BuiltinType.AINT32, BuiltinType.ASTRING, BuiltinType.AINT32 };
+ ARecordType userRecordType = new ARecordType("TwitterUserType", userFieldNames, userFieldTypes, false);
+
+ String[] fieldNames = new String[] { "tweetid", "user", "sender-location", "send-time", "referred-topics",
+ "message-text" };
+
+ AUnorderedListType unorderedListType = new AUnorderedListType(BuiltinType.ASTRING, "referred-topics");
+ IAType[] fieldTypes = new IAType[] { BuiltinType.ASTRING, userRecordType, BuiltinType.APOINT,
+ BuiltinType.ADATETIME, unorderedListType, BuiltinType.ASTRING };
+ outputType = new ARecordType("TweetMessageType", fieldNames, fieldTypes, false);
+
+ } catch (AsterixException e) {
+ throw new IllegalStateException("Unable to initialize output type");
+ }
+ return outputType;
+ }
+}
\ No newline at end of file
diff --git a/asterix-tools/src/main/resources/test.properties b/asterix-tools/src/main/resources/test.properties
old mode 100755
new mode 100644
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
index 933afcd..419438f 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
@@ -134,7 +134,7 @@
appendPage.append(logRecord, appendLSN);
appendLSN += logRecord.getLogSize();
}
-
+
private void getAndInitNewPage() {
appendPage = null;
while (appendPage == null) {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
index 6fb91c8..06f5399 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
@@ -43,190 +43,203 @@
*/
public class TransactionContext implements ITransactionContext, Serializable {
- private static final long serialVersionUID = -6105616785783310111L;
- private TransactionSubsystem transactionSubsystem;
+ private static final long serialVersionUID = -6105616785783310111L;
+ private TransactionSubsystem transactionSubsystem;
- //jobId is set once and read concurrently.
- private final JobId jobId;
+ // jobId is set once and read concurrently.
+ private final JobId jobId;
- //There are no concurrent writers on both firstLSN and lastLSN
- //since both values are updated by serialized log appenders.
- //But readers and writers can be different threads,
- //so both LSNs are atomic variables in order to be read and written atomically.
- private AtomicLong firstLSN;
- private AtomicLong lastLSN;
+ // There are no concurrent writers on both firstLSN and lastLSN
+ // since both values are updated by serialized log appenders.
+ // But readers and writers can be different threads,
+ // so both LSNs are atomic variables in order to be read and written
+ // atomically.
+ private AtomicLong firstLSN;
+ private AtomicLong lastLSN;
- //txnState is read and written concurrently.
- private AtomicInteger txnState;
+ // txnState is read and written concurrently.
+ private AtomicInteger txnState;
- //isTimeout is read and written under the lockMgr's tableLatch
- //Thus, no other synchronization is required separately.
- private boolean isTimeout;
+ // isTimeout is read and written under the lockMgr's tableLatch
+ // Thus, no other synchronization is required separately.
+ private boolean isTimeout;
- //isWriteTxn can be set concurrently by multiple threads.
- private AtomicBoolean isWriteTxn;
+ // isWriteTxn can be set concurrently by multiple threads.
+ private AtomicBoolean isWriteTxn;
- //isMetadataTxn is accessed by a single thread since the metadata is not partitioned
- private boolean isMetadataTxn;
+ // isMetadataTxn is accessed by a single thread since the metadata is not
+ // partitioned
+ private boolean isMetadataTxn;
- //indexMap is concurrently accessed by multiple threads,
- //so those threads are synchronized on indexMap object itself
- private Map<MutableLong, AbstractLSMIOOperationCallback> indexMap;
+ // indexMap is concurrently accessed by multiple threads,
+ // so those threads are synchronized on indexMap object itself
+ private Map<MutableLong, AbstractLSMIOOperationCallback> indexMap;
- //TODO: fix ComponentLSNs' issues.
- //primaryIndex, primaryIndexCallback, and primaryIndexOptracker will be modified accordingly
- //when the issues of componentLSNs are fixed.
- private ILSMIndex primaryIndex;
- private PrimaryIndexModificationOperationCallback primaryIndexCallback;
- private PrimaryIndexOperationTracker primaryIndexOpTracker;
+ // TODO: fix ComponentLSNs' issues.
+ // primaryIndex, primaryIndexCallback, and primaryIndexOptracker will be
+ // modified accordingly
+ // when the issues of componentLSNs are fixed.
+ private ILSMIndex primaryIndex;
+ private PrimaryIndexModificationOperationCallback primaryIndexCallback;
+ private PrimaryIndexOperationTracker primaryIndexOpTracker;
- //The following three variables are used as temporary variables in order to avoid object creations.
- //Those are used in synchronized methods.
- private MutableLong tempResourceIdForRegister;
- private MutableLong tempResourceIdForSetLSN;
- private LogRecord logRecord;
+ // The following three variables are used as temporary variables in order to
+ // avoid object creations.
+ // Those are used in synchronized methods.
+ private MutableLong tempResourceIdForRegister;
+ private MutableLong tempResourceIdForSetLSN;
+ private LogRecord logRecord;
- //TODO: implement transactionContext pool in order to avoid object creations.
- // also, the pool can throttle the number of concurrent active jobs at every moment.
- public TransactionContext(JobId jobId, TransactionSubsystem transactionSubsystem) throws ACIDException {
- this.jobId = jobId;
- this.transactionSubsystem = transactionSubsystem;
- firstLSN = new AtomicLong(-1);
- lastLSN = new AtomicLong(-1);
- txnState = new AtomicInteger(ITransactionManager.ACTIVE);
- isTimeout = false;
- isWriteTxn = new AtomicBoolean(false);
- isMetadataTxn = false;
- indexMap = new HashMap<MutableLong, AbstractLSMIOOperationCallback>();
- primaryIndex = null;
- tempResourceIdForRegister = new MutableLong();
- tempResourceIdForSetLSN = new MutableLong();
- logRecord = new LogRecord();
- }
+ // TODO: implement transactionContext pool in order to avoid object
+ // creations.
+ // also, the pool can throttle the number of concurrent active jobs at every
+ // moment.
+ public TransactionContext(JobId jobId,
+ TransactionSubsystem transactionSubsystem) throws ACIDException {
+ this.jobId = jobId;
+ this.transactionSubsystem = transactionSubsystem;
+ firstLSN = new AtomicLong(-1);
+ lastLSN = new AtomicLong(-1);
+ txnState = new AtomicInteger(ITransactionManager.ACTIVE);
+ isTimeout = false;
+ isWriteTxn = new AtomicBoolean(false);
+ isMetadataTxn = false;
+ indexMap = new HashMap<MutableLong, AbstractLSMIOOperationCallback>();
+ primaryIndex = null;
+ tempResourceIdForRegister = new MutableLong();
+ tempResourceIdForSetLSN = new MutableLong();
+ logRecord = new LogRecord();
+ }
- public void registerIndexAndCallback(long resourceId, ILSMIndex index, AbstractOperationCallback callback,
- boolean isPrimaryIndex) {
- synchronized (indexMap) {
- if (isPrimaryIndex && primaryIndex == null) {
- primaryIndex = index;
- primaryIndexCallback = (PrimaryIndexModificationOperationCallback) callback;
- primaryIndexOpTracker = (PrimaryIndexOperationTracker) index.getOperationTracker();
- }
- tempResourceIdForRegister.set(resourceId);
- if (!indexMap.containsKey(tempResourceIdForRegister)) {
- indexMap.put(new MutableLong(resourceId),
- ((AbstractLSMIOOperationCallback) index.getIOOperationCallback()));
- }
- }
- }
+ public void registerIndexAndCallback(long resourceId, ILSMIndex index,
+ AbstractOperationCallback callback, boolean isPrimaryIndex) {
+ synchronized (indexMap) {
+ if (isPrimaryIndex && primaryIndex == null) {
+ primaryIndex = index;
+ primaryIndexCallback = (PrimaryIndexModificationOperationCallback) callback;
+ primaryIndexOpTracker = (PrimaryIndexOperationTracker) index
+ .getOperationTracker();
+ }
+ tempResourceIdForRegister.set(resourceId);
+ if (!indexMap.containsKey(tempResourceIdForRegister)) {
+ indexMap.put(new MutableLong(resourceId),
+ ((AbstractLSMIOOperationCallback) index
+ .getIOOperationCallback()));
+ }
+ }
+ }
- //[Notice]
- //This method is called sequentially by the LogAppender threads.
- //However, the indexMap is concurrently read and modified through this method and registerIndexAndCallback()
- @Override
- public void setLastLSN(long resourceId, long LSN) {
- synchronized (indexMap) {
- firstLSN.compareAndSet(-1, LSN);
- lastLSN.set(Math.max(lastLSN.get(), LSN));
- if (resourceId != -1) {
- //Non-update log's resourceId is -1.
- tempResourceIdForSetLSN.set(resourceId);
- AbstractLSMIOOperationCallback ioOpCallback = indexMap.get(tempResourceIdForSetLSN);
- ioOpCallback.updateLastLSN(LSN);
- }
- }
- }
+ // [Notice]
+ // This method is called sequentially by the LogAppender threads.
+ // However, the indexMap is concurrently read and modified through this
+ // method and registerIndexAndCallback()
+ @Override
+ public void setLastLSN(long resourceId, long LSN) {
+ synchronized (indexMap) {
+ firstLSN.compareAndSet(-1, LSN);
+ lastLSN.set(Math.max(lastLSN.get(), LSN));
+ if (resourceId != -1) {
+ // Non-update log's resourceId is -1.
+ tempResourceIdForSetLSN.set(resourceId);
+ AbstractLSMIOOperationCallback ioOpCallback = indexMap
+ .get(tempResourceIdForSetLSN);
+ ioOpCallback.updateLastLSN(LSN);
+ }
+ }
+ }
- @Override
- public void notifyOptracker(boolean isJobLevelCommit) {
- try {
- if (isJobLevelCommit && isMetadataTxn) {
- primaryIndexOpTracker.exclusiveJobCommitted();
- } else if (!isJobLevelCommit) {
- primaryIndexOpTracker
- .completeOperation(null, LSMOperationType.MODIFICATION, null, primaryIndexCallback);
- }
- } catch (HyracksDataException e) {
- throw new IllegalStateException(e);
- }
- }
+ @Override
+ public void notifyOptracker(boolean isJobLevelCommit) {
+ try {
+ if (isJobLevelCommit && isMetadataTxn) {
+ primaryIndexOpTracker.exclusiveJobCommitted();
+ } else if (!isJobLevelCommit) {
+ primaryIndexOpTracker.completeOperation(null,
+ LSMOperationType.MODIFICATION, null,
+ primaryIndexCallback);
+ }
+ } catch (HyracksDataException e) {
+ throw new IllegalStateException(e);
+ }
+ }
- public void setWriteTxn(boolean isWriteTxn) {
- this.isWriteTxn.set(isWriteTxn);
- }
+ public void setWriteTxn(boolean isWriteTxn) {
+ this.isWriteTxn.set(isWriteTxn);
+ }
- public boolean isWriteTxn() {
- return isWriteTxn.get();
- }
+ public boolean isWriteTxn() {
+ return isWriteTxn.get();
+ }
- @Override
- public long getFirstLSN() {
- return firstLSN.get();
- }
+ @Override
+ public long getFirstLSN() {
+ return firstLSN.get();
+ }
- @Override
- public long getLastLSN() {
- return lastLSN.get();
- }
+ @Override
+ public long getLastLSN() {
+ return lastLSN.get();
+ }
- public JobId getJobId() {
- return jobId;
- }
+ public JobId getJobId() {
+ return jobId;
+ }
- public void setTimeout(boolean isTimeout) {
- this.isTimeout = isTimeout;
- }
+ public void setTimeout(boolean isTimeout) {
+ this.isTimeout = isTimeout;
+ }
- public boolean isTimeout() {
- return isTimeout;
- }
+ public boolean isTimeout() {
+ return isTimeout;
+ }
- public void setTxnState(int txnState) {
- this.txnState.set(txnState);
- }
+ public void setTxnState(int txnState) {
+ this.txnState.set(txnState);
+ }
- public int getTxnState() {
- return txnState.get();
- }
+ public int getTxnState() {
+ return txnState.get();
+ }
- @Override
- public int hashCode() {
- return jobId.getId();
- }
+ @Override
+ public int hashCode() {
+ return jobId.getId();
+ }
- @Override
- public boolean equals(Object o) {
- return (o == this);
- }
+ @Override
+ public boolean equals(Object o) {
+ return (o == this);
+ }
- @Override
- public void setMetadataTransaction(boolean isMetadataTxn) {
- this.isMetadataTxn = isMetadataTxn;
- }
+ @Override
+ public void setMetadataTransaction(boolean isMetadataTxn) {
+ this.isMetadataTxn = isMetadataTxn;
+ }
- @Override
- public boolean isMetadataTransaction() {
- return isMetadataTxn;
- }
+ @Override
+ public boolean isMetadataTransaction() {
+ return isMetadataTxn;
+ }
- public String prettyPrint() {
- StringBuilder sb = new StringBuilder();
- sb.append("\n" + jobId + "\n");
- sb.append("isWriteTxn: " + isWriteTxn + "\n");
- sb.append("firstLSN: " + firstLSN.get() + "\n");
- sb.append("lastLSN: " + lastLSN.get() + "\n");
- sb.append("TransactionState: " + txnState + "\n");
- sb.append("isTimeout: " + isTimeout + "\n");
- return sb.toString();
- }
+ public String prettyPrint() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("\n" + jobId + "\n");
+ sb.append("isWriteTxn: " + isWriteTxn + "\n");
+ sb.append("firstLSN: " + firstLSN.get() + "\n");
+ sb.append("lastLSN: " + lastLSN.get() + "\n");
+ sb.append("TransactionState: " + txnState + "\n");
+ sb.append("isTimeout: " + isTimeout + "\n");
+ return sb.toString();
+ }
- public LogRecord getLogRecord() {
- return logRecord;
- }
+ public LogRecord getLogRecord() {
+ return logRecord;
+ }
- public void cleanupForAbort() {
- if (primaryIndexOpTracker != null) {
- primaryIndexOpTracker.cleanupNumActiveOperationsForAbortedJob(primaryIndexCallback);
- }
- }
+ public void cleanupForAbort() {
+ if (primaryIndexOpTracker != null) {
+ primaryIndexOpTracker
+ .cleanupNumActiveOperationsForAbortedJob(primaryIndexCallback);
+ }
+ }
}